You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/06/16 21:26:23 UTC

svn commit: r1602968 [5/5] - in /lucene/dev/branches/lucene5752: ./ dev-tools/ dev-tools/idea/.idea/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/ lucene/codecs/ lucene/codecs/src/...

Modified: lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (original)
+++ lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java Mon Jun 16 19:26:19 2014
@@ -21,6 +21,7 @@ import static org.apache.lucene.index.So
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -38,6 +39,7 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FloatDocValuesField;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.StringField;
@@ -79,6 +81,12 @@ public abstract class BaseDocValuesForma
         doc.add(new SortedSetDocValuesField("ssdv", new BytesRef(TestUtil.randomSimpleString(random(), 2))));
       }
     }
+    if (defaultCodecSupportsSortedNumeric()) {
+      final int numValues = random().nextInt(5);
+      for (int i = 0; i < numValues; ++i) {
+        doc.add(new SortedNumericDocValuesField("sndv", TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE)));
+      }
+    }
   }
 
   public void testOneNumber() throws IOException {
@@ -1193,6 +1201,69 @@ public abstract class BaseDocValuesForma
     dir.close();
   }
   
+  private void doTestSortedNumericsVsStoredFields(LongProducer counts, LongProducer values) throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+    
+    // index some docs
+    int numDocs = atLeast(300);
+    // numDocs should be always > 256 so that in case of a codec that optimizes
+    // for numbers of values <= 256, all storage layouts are tested
+    assert numDocs > 256;
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      doc.add(new StringField("id", Integer.toString(i), Field.Store.NO));
+      
+      int valueCount = (int) counts.next();
+      long valueArray[] = new long[valueCount];
+      for (int j = 0; j < valueCount; j++) {
+        long value = values.next();
+        valueArray[j] = value;
+        doc.add(new SortedNumericDocValuesField("dv", value));
+      }
+      Arrays.sort(valueArray);
+      for (int j = 0; j < valueCount; j++) {
+        doc.add(new StoredField("stored", Long.toString(valueArray[j])));
+      }
+      writer.addDocument(doc);
+      if (random().nextInt(31) == 0) {
+        writer.commit();
+      }
+    }
+    
+    // delete some docs
+    int numDeletions = random().nextInt(numDocs/10);
+    for (int i = 0; i < numDeletions; i++) {
+      int id = random().nextInt(numDocs);
+      writer.deleteDocuments(new Term("id", Integer.toString(id)));
+    }
+
+    // merge some segments and ensure that at least one of them has more than
+    // 256 values
+    writer.forceMerge(numDocs / 256);
+
+    writer.shutdown();
+    
+    // compare
+    DirectoryReader ir = DirectoryReader.open(dir);
+    for (AtomicReaderContext context : ir.leaves()) {
+      AtomicReader r = context.reader();
+      SortedNumericDocValues docValues = DocValues.getSortedNumeric(r, "dv");
+      for (int i = 0; i < r.maxDoc(); i++) {
+        String expected[] = r.document(i).getValues("stored");
+        docValues.setDocument(i);
+        String actual[] = new String[docValues.count()];
+        for (int j = 0; j < actual.length; j++) {
+          actual[j] = Long.toString(docValues.valueAt(j));
+        }
+        assertArrayEquals(expected, actual);
+      }
+    }
+    ir.close();
+    dir.close();
+  }
+  
   public void testBooleanNumericsVsStoredFields() throws Exception {
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
@@ -1889,6 +1960,69 @@ public abstract class BaseDocValuesForma
     }
   }
   
+  public void testSortedNumericsSingleValuedVsStoredFields() throws Exception {
+    assumeTrue("Codec does not support SORTED_NUMERIC", defaultCodecSupportsSortedNumeric());
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedNumericsVsStoredFields(
+          new LongProducer() {
+            @Override
+            long next() {
+              return 1;
+            }
+          },
+          new LongProducer() {
+            @Override
+            long next() {
+              return TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE);
+            }
+          }
+      );
+    }
+  }
+  
+  public void testSortedNumericsSingleValuedMissingVsStoredFields() throws Exception {
+    assumeTrue("Codec does not support SORTED_NUMERIC", defaultCodecSupportsSortedNumeric());
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedNumericsVsStoredFields(
+          new LongProducer() {
+            @Override
+            long next() {
+              return random().nextBoolean() ? 0 : 1;
+            }
+          },
+          new LongProducer() {
+            @Override
+            long next() {
+              return TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE);
+            }
+          }
+      );
+    }
+  }
+  
+  public void testSortedNumericsMultipleValuesVsStoredFields() throws Exception {
+    assumeTrue("Codec does not support SORTED_NUMERIC", defaultCodecSupportsSortedNumeric());
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedNumericsVsStoredFields(
+          new LongProducer() {
+            @Override
+            long next() {
+              return TestUtil.nextLong(random(), 0, 50);
+            }
+          },
+          new LongProducer() {
+            @Override
+            long next() {
+              return TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE);
+            }
+          }
+      );
+    }
+  }
+  
   public void testSortedSetVariableLengthVsStoredFields() throws Exception {
     assumeTrue("Codec does not support SORTED_SET", defaultCodecSupportsSortedSet());
     int numIterations = atLeast(1);
@@ -2238,6 +2372,7 @@ public abstract class BaseDocValuesForma
   public void testThreads2() throws Exception {
     assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
     assumeTrue("Codec does not support SORTED_SET", defaultCodecSupportsSortedSet());
+    assumeTrue("Codec does not support SORTED_NUMERIC", defaultCodecSupportsSortedNumeric());
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
@@ -2281,6 +2416,15 @@ public abstract class BaseDocValuesForma
         doc.add(new SortedSetDocValuesField("dvSortedSet", new BytesRef(v)));
         doc.add(new StoredField("storedSortedSet", v));
       }
+      int numSortedNumericFields = random().nextInt(3);
+      Set<Long> numValues = new TreeSet<>();
+      for (int j = 0; j < numSortedNumericFields; j++) {
+        numValues.add(TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE));
+      }
+      for (Long l : numValues) {
+        doc.add(new SortedNumericDocValuesField("dvSortedNumeric", l));
+        doc.add(new StoredField("storedSortedNumeric", Long.toString(l)));
+      }
       writer.addDocument(doc);
       if (random().nextInt(31) == 0) {
         writer.commit();
@@ -2317,6 +2461,8 @@ public abstract class BaseDocValuesForma
               Bits numericBits = r.getDocsWithField("dvNum");
               SortedSetDocValues sortedSet = r.getSortedSetDocValues("dvSortedSet");
               Bits sortedSetBits = r.getDocsWithField("dvSortedSet");
+              SortedNumericDocValues sortedNumeric = r.getSortedNumericDocValues("dvSortedNumeric");
+              Bits sortedNumericBits = r.getDocsWithField("dvSortedNumeric");
               for (int j = 0; j < r.maxDoc(); j++) {
                 BytesRef binaryValue = r.document(j).getBinaryValue("storedBin");
                 if (binaryValue != null) {
@@ -2361,6 +2507,22 @@ public abstract class BaseDocValuesForma
                   assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSet.nextOrd());
                   assertFalse(sortedSetBits.get(j));
                 }
+                
+                String numValues[] = r.document(j).getValues("storedSortedNumeric");
+                if (numValues.length > 0) {
+                  assertNotNull(sortedNumeric);
+                  sortedNumeric.setDocument(j);
+                  assertEquals(numValues.length, sortedNumeric.count());
+                  for (int k = 0; k < numValues.length; k++) {
+                    long v = sortedNumeric.valueAt(k);
+                    assertEquals(numValues[k], Long.toString(v));
+                  }
+                  assertTrue(sortedNumericBits.get(j));
+                } else if (sortedNumeric != null) {
+                  sortedNumeric.setDocument(j);
+                  assertEquals(0, sortedNumeric.count());
+                  assertFalse(sortedNumericBits.get(j));
+                }
               }
             }
             TestUtil.checkReader(ir);
@@ -2414,6 +2576,171 @@ public abstract class BaseDocValuesForma
       dir.close();
     }
   }
+  
+  public void testOneSortedNumber() throws IOException {
+    assumeTrue("Codec does not support SORTED_NUMERIC", defaultCodecSupportsSortedNumeric());
+    Directory directory = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory);
+    Document doc = new Document();
+    doc.add(new SortedNumericDocValuesField("dv", 5));
+    writer.addDocument(doc);
+    writer.shutdown();
+    
+    // Now search the index:
+    IndexReader reader = DirectoryReader.open(directory);
+    assert reader.leaves().size() == 1;
+    SortedNumericDocValues dv = reader.leaves().get(0).reader().getSortedNumericDocValues("dv");
+    dv.setDocument(0);
+    assertEquals(1, dv.count());
+    assertEquals(5, dv.valueAt(0));
+
+    reader.close();
+    directory.close();
+  }
+  
+  public void testOneSortedNumberOneMissing() throws IOException {
+    assumeTrue("Codec does not support SORTED_NUMERIC", defaultCodecSupportsSortedNumeric());
+    Directory directory = newDirectory();
+    IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(TEST_VERSION_CURRENT, null));
+    Document doc = new Document();
+    doc.add(new SortedNumericDocValuesField("dv", 5));
+    writer.addDocument(doc);
+    writer.addDocument(new Document());
+    writer.shutdown();
+    
+    // Now search the index:
+    IndexReader reader = DirectoryReader.open(directory);
+    assert reader.leaves().size() == 1;
+    SortedNumericDocValues dv = reader.leaves().get(0).reader().getSortedNumericDocValues("dv");
+    dv.setDocument(0);
+    assertEquals(1, dv.count());
+    assertEquals(5, dv.valueAt(0));
+    dv.setDocument(1);
+    assertEquals(0, dv.count());
+    
+    Bits docsWithField = reader.leaves().get(0).reader().getDocsWithField("dv");
+    assertEquals(2, docsWithField.length());
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+
+    reader.close();
+    directory.close();
+  }
+  
+  public void testTwoSortedNumber() throws IOException {
+    assumeTrue("Codec does not support SORTED_NUMERIC", defaultCodecSupportsSortedNumeric());
+    Directory directory = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory);
+    Document doc = new Document();
+    doc.add(new SortedNumericDocValuesField("dv", 11));
+    doc.add(new SortedNumericDocValuesField("dv", -5));
+    writer.addDocument(doc);
+    writer.shutdown();
+    
+    // Now search the index:
+    IndexReader reader = DirectoryReader.open(directory);
+    assert reader.leaves().size() == 1;
+    SortedNumericDocValues dv = reader.leaves().get(0).reader().getSortedNumericDocValues("dv");
+    dv.setDocument(0);
+    assertEquals(2, dv.count());
+    assertEquals(-5, dv.valueAt(0));
+    assertEquals(11, dv.valueAt(1));
+
+    reader.close();
+    directory.close();
+  }
+  
+  public void testTwoSortedNumberOneMissing() throws IOException {
+    assumeTrue("Codec does not support SORTED_NUMERIC", defaultCodecSupportsSortedNumeric());
+    Directory directory = newDirectory();
+    IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(TEST_VERSION_CURRENT, null));
+    Document doc = new Document();
+    doc.add(new SortedNumericDocValuesField("dv", 11));
+    doc.add(new SortedNumericDocValuesField("dv", -5));
+    writer.addDocument(doc);
+    writer.addDocument(new Document());
+    writer.shutdown();
+    
+    // Now search the index:
+    IndexReader reader = DirectoryReader.open(directory);
+    assert reader.leaves().size() == 1;
+    SortedNumericDocValues dv = reader.leaves().get(0).reader().getSortedNumericDocValues("dv");
+    dv.setDocument(0);
+    assertEquals(2, dv.count());
+    assertEquals(-5, dv.valueAt(0));
+    assertEquals(11, dv.valueAt(1));
+    dv.setDocument(1);
+    assertEquals(0, dv.count());
+    
+    Bits docsWithField = reader.leaves().get(0).reader().getDocsWithField("dv");
+    assertEquals(2, docsWithField.length());
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+
+    reader.close();
+    directory.close();
+  }
+  
+  public void testSortedNumberMerge() throws IOException {
+    assumeTrue("Codec does not support SORTED_NUMERIC", defaultCodecSupportsSortedNumeric());
+    Directory directory = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(TEST_VERSION_CURRENT, null);
+    iwc.setMergePolicy(newLogMergePolicy());
+    IndexWriter writer = new IndexWriter(directory, iwc);
+    Document doc = new Document();
+    doc.add(new SortedNumericDocValuesField("dv", 11));
+    writer.addDocument(doc);
+    writer.commit();
+    doc = new Document();
+    doc.add(new SortedNumericDocValuesField("dv", -5));
+    writer.addDocument(doc);
+    writer.forceMerge(1);
+    writer.shutdown();
+    
+    // Now search the index:
+    IndexReader reader = DirectoryReader.open(directory);
+    assert reader.leaves().size() == 1;
+    SortedNumericDocValues dv = reader.leaves().get(0).reader().getSortedNumericDocValues("dv");
+    dv.setDocument(0);
+    assertEquals(1, dv.count());
+    assertEquals(11, dv.valueAt(0));
+    dv.setDocument(1);
+    assertEquals(1, dv.count());
+    assertEquals(-5, dv.valueAt(0));
+
+    reader.close();
+    directory.close();
+  }
+  
+  public void testSortedNumberMergeAwayAllValues() throws IOException {
+    assumeTrue("Codec does not support SORTED_NUMERIC", defaultCodecSupportsSortedNumeric());
+    Directory directory = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwconfig = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwconfig.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iwriter = new RandomIndexWriter(random(), directory, iwconfig);
+    
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.NO));
+    iwriter.addDocument(doc);    
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.NO));
+    doc.add(new SortedNumericDocValuesField("field", 5));
+    iwriter.addDocument(doc);
+    iwriter.commit();
+    iwriter.deleteDocuments(new Term("id", "1"));
+    iwriter.forceMerge(1);
+    
+    DirectoryReader ireader = iwriter.getReader();
+    iwriter.shutdown();
+    
+    SortedNumericDocValues dv = getOnlySegmentReader(ireader).getSortedNumericDocValues("field");
+    dv.setDocument(0);
+    assertEquals(0, dv.count());
+    
+    ireader.close();
+    directory.close();
+  }
 
   protected boolean codecAcceptsHugeBinaryValues(String field) {
     return true;

Modified: lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Mon Jun 16 19:26:19 2014
@@ -59,6 +59,7 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.RamUsageTester;
 import org.apache.lucene.util.TestUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -1566,7 +1567,7 @@ public abstract class BasePostingsFormat
     while (bytesIndexed < bytesToIndex) {
       Document doc = docs.nextDoc();
       w.addDocument(doc);
-      bytesIndexed += RamUsageEstimator.sizeOf(doc);
+      bytesIndexed += RamUsageTester.sizeOf(doc);
     }
 
     IndexReader r = w.getReader();

Modified: lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java Mon Jun 16 19:26:19 2014
@@ -133,6 +133,11 @@ public final class FieldFilterAtomicRead
   }
   
   @Override
+  public SortedNumericDocValues getSortedNumericDocValues(String field) throws IOException {
+    return hasField(field) ? super.getSortedNumericDocValues(field) : null;
+  }
+  
+  @Override
   public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
     return hasField(field) ? super.getSortedSetDocValues(field) : null;
   }

Modified: lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java Mon Jun 16 19:26:19 2014
@@ -32,7 +32,6 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
 import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
 import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
-import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
 import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
 import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
@@ -151,7 +150,6 @@ public class RandomCodec extends Lucene4
     
     addDocValues(avoidCodecs,
         new Lucene49DocValuesFormat(),
-        new DiskDocValuesFormat(),
         new MemoryDocValuesFormat(),
         new SimpleTextDocValuesFormat(),
         new AssertingDocValuesFormat());

Modified: lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/util/BaseDocIdSetTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/util/BaseDocIdSetTestCase.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/util/BaseDocIdSetTestCase.java (original)
+++ lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/util/BaseDocIdSetTestCase.java Mon Jun 16 19:26:19 2014
@@ -113,6 +113,21 @@ public abstract class BaseDocIdSetTestCa
     }
   }
 
+  /** Test ram usage estimation. */
+  public void testRamBytesUsed() throws IOException {
+    final int iters = 100;
+    for (int i = 0; i < iters; ++i) {
+      final int pow = random().nextInt(20);
+      final int maxDoc = TestUtil.nextInt(random(), 1, 1 << pow);
+      final int numDocs = TestUtil.nextInt(random(), 0, Math.min(maxDoc, 1 << TestUtil.nextInt(random(), 0, pow)));
+      final BitSet set = randomSet(maxDoc, numDocs);
+      final DocIdSet copy = copyOf(set, maxDoc);
+      final long actualBytes = ramBytesUsed(copy, maxDoc);
+      final long expectedBytes = copy.ramBytesUsed();
+      assertEquals(expectedBytes, actualBytes);
+    }
+  }
+
   /** Assert that the content of the {@link DocIdSet} is the same as the content of the {@link BitSet}. */
   public void assertEquals(int numBits, BitSet ds1, T ds2) throws IOException {
     // nextDoc
@@ -172,4 +187,21 @@ public abstract class BaseDocIdSetTestCa
     }
   }
 
+  private static class Dummy {
+    @SuppressWarnings("unused")
+    Object o1, o2;
+  }
+
+  // same as RamUsageTester.sizeOf but tries to not take into account resources
+  // that might be shared across instances
+  private long ramBytesUsed(DocIdSet set, int length) throws IOException {
+    Dummy dummy = new Dummy();
+    dummy.o1 = copyOf(new BitSet(length), length);
+    dummy.o2 = set;
+    long bytes1 = RamUsageTester.sizeOf(dummy);
+    dummy.o2 = null;
+    long bytes2 = RamUsageTester.sizeOf(dummy);
+    return bytes1 - bytes2;
+  }
+
 }

Modified: lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/lucene5752/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Mon Jun 16 19:26:19 2014
@@ -95,6 +95,7 @@ import org.apache.lucene.index.SerialMer
 import org.apache.lucene.index.SimpleMergedSegmentWarmer;
 import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.index.StoredDocument;
@@ -1680,6 +1681,15 @@ public abstract class LuceneTestCase ext
     return true;
   }
   
+  /** Returns true if the default codec supports SORTED_NUMERIC docvalues */ 
+  public static boolean defaultCodecSupportsSortedNumeric() {
+    String name = Codec.getDefault().getName();
+    if (name.equals("Lucene40") || name.equals("Lucene41") || name.equals("Lucene42") || name.equals("Lucene45") || name.equals("Lucene46")) {
+      return false;
+    }
+    return true;
+  }
+  
   /** Returns true if the codec "supports" docsWithField 
    * (other codecs return MatchAllBits, because you couldnt write missing values before) */
   public static boolean defaultCodecSupportsDocsWithField() {
@@ -2289,6 +2299,28 @@ public abstract class LuceneTestCase ext
       }
       
       {
+        SortedNumericDocValues leftValues = MultiDocValues.getSortedNumericValues(leftReader, field);
+        SortedNumericDocValues rightValues = MultiDocValues.getSortedNumericValues(rightReader, field);
+        if (leftValues != null && rightValues != null) {
+          for (int i = 0; i < leftReader.maxDoc(); i++) {
+            leftValues.setDocument(i);
+            long expected[] = new long[leftValues.count()];
+            for (int j = 0; j < expected.length; j++) {
+              expected[j] = leftValues.valueAt(j);
+            }
+            rightValues.setDocument(i);
+            for (int j = 0; j < expected.length; j++) {
+              assertEquals(info, expected[j], rightValues.valueAt(j));
+            }
+            assertEquals(info, expected.length, rightValues.count());
+          }
+        } else {
+          assertNull(info, leftValues);
+          assertNull(info, rightValues);
+        }
+      }
+      
+      {
         Bits leftBits = MultiDocValues.getDocsWithField(leftReader, field);
         Bits rightBits = MultiDocValues.getDocsWithField(rightReader, field);
         if (leftBits != null && rightBits != null) {

Modified: lucene/dev/branches/lucene5752/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/CHANGES.txt?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/CHANGES.txt (original)
+++ lucene/dev/branches/lucene5752/solr/CHANGES.txt Mon Jun 16 19:26:19 2014
@@ -84,8 +84,7 @@ Other Changes
 * SOLR-5868: HttpClient should be configured to use ALLOW_ALL_HOSTNAME hostname
   verifier to simplify SSL setup. (Steve Davids via Mark Miller)
 
-* SOLR-6015: Better way to handle managed synonyms when ignoreCase=true
-  (Timothy Potter)
+* SOLR-6169: Properly remove CoreAdminHandler handleAlias action (Alan Woodward)
 
 ==================  4.9.0 ==================
 
@@ -139,10 +138,17 @@ New Features
 Bug Fixes
 ----------------------
 
+* SOLR-5956: Use coreDescriptor.getInstanceDir() instead of getRawInstanceDir()
+  in the SnapShooter to avoid problems when solr.solr.home is a symbolic link.
+  (Timothy Potter) 
+
 * SOLR-6002: Fix a couple of ugly issues around SolrIndexWriter close and 
   rollback as well as how SolrIndexWriter manages it's ref counted directory
   instance. (Mark Miller, Gregory Chanan)
 
+* SOLR-6015: Better way to handle managed synonyms when ignoreCase=true
+  (Timothy Potter)
+
 * SOLR-6104: The 'addreplica' Collection API does not support 'async' parameter.
   (shalin)
 
@@ -172,6 +178,12 @@ Bug Fixes
 * SOLR-6158: Relative configSetBase directories were resolved relative to the 
   container CWD, rather than solr.home. (Simon Endele, Alan Woodward)
 
+* SOLR-6056: Don't publish recovery state until recovery runs to avoid overwhelming
+  the overseer state queue. (Raintung Li, Mark Miller, shalin)
+
+* SOLR-5426: Fixed a bug in ReverseWildCardFilter that could cause 
+  InvalidTokenOffsetsException when highlighting. (Uwe Schindler, Arun Kumar, via hossman)
+
 Other Changes
 ---------------------
 
@@ -215,6 +227,15 @@ Other Changes
 * SOLR-6067: Refactor duplicate Collector code in SolrIndexSearcher 
   (Christine Poerschke via hossman)
 
+* SOLR-5940: post.jar reports back detailed error in case of error responses.
+  (Sameer Maggon, shalin, Uwe Schindler)
+
+* SOLR-6161: SolrDispatchFilter should throw java.lang.Error back even if wrapped in
+  another exception. (Miklos Christine via shalin)
+
+* SOLR-6153: ReplicationHandler backup response format should contain backup name.
+  (Varun Thacker via shalin)
+
 Optimizations
 ----------------------
 

Modified: lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/analysis/ReversedWildcardFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/analysis/ReversedWildcardFilter.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/analysis/ReversedWildcardFilter.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/analysis/ReversedWildcardFilter.java Mon Jun 16 19:26:19 2014
@@ -37,11 +37,11 @@ import org.apache.lucene.analysis.tokena
  */
 public final class ReversedWildcardFilter extends TokenFilter {
   
-  private boolean withOriginal;
-  private char markerChar;
-  private State save;
-  private CharTermAttribute termAtt;
-  private PositionIncrementAttribute posAtt;
+  private final boolean withOriginal;
+  private final char markerChar;
+  private final CharTermAttribute termAtt;
+  private final PositionIncrementAttribute posAtt;
+  private State save = null;
 
   protected ReversedWildcardFilter(TokenStream input, boolean withOriginal, char markerChar) {
     super(input);
@@ -146,5 +146,11 @@ public final class ReversedWildcardFilte
       buffer[end] = allowFrontSur ? endLow : frontHigh;
     }
   }
+  
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    save = null;
+  }
 
 }

Modified: lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/handler/SnapShooter.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/handler/SnapShooter.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/handler/SnapShooter.java Mon Jun 16 19:26:19 2014
@@ -62,7 +62,7 @@ public class SnapShooter {
     solrCore = core;
     if (location == null) snapDir = core.getDataDir();
     else  {
-      File base = new File(core.getCoreDescriptor().getRawInstanceDir());
+      File base = new File(core.getCoreDescriptor().getInstanceDir());
       snapDir = org.apache.solr.util.FileUtils.resolvePath(base, location).getAbsolutePath();
       File dir = new File(snapDir);
       if (!dir.exists())  dir.mkdirs();
@@ -154,6 +154,8 @@ public class SnapShooter {
       details.add("fileCount", files.size());
       details.add("status", "success");
       details.add("snapshotCompletedAt", new Date().toString());
+      details.add("snapshotName", snapshotName);
+      LOG.info("Done creating backup snapshot: " + (snapshotName == null ? "<not named>" : snapshotName));
     } catch (Exception e) {
       SnapPuller.delTree(snapShotDir);
       LOG.error("Exception while creating snapshot", e);
@@ -203,6 +205,7 @@ public class SnapShooter {
 
     if(isSuccess) {
       details.add("status", "success");
+      details.add("snapshotDeletedAt", new Date().toString());
     } else {
       details.add("status", "Unable to delete snapshot: " + snapshotName);
       LOG.warn("Unable to delete snapshot: " + snapshotName);

Modified: lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java Mon Jun 16 19:26:19 2014
@@ -625,28 +625,6 @@ public class CoreAdminHandler extends Re
   }
 
   /**
-   * Handle "ALIAS" action
-   */
-  @Deprecated
-  protected void handleAliasAction(SolrQueryRequest req, SolrQueryResponse rsp) {
-    SolrParams params = req.getParams();
-
-    String name = params.get(CoreAdminParams.OTHER);
-    String cname = params.get(CoreAdminParams.CORE);
-    boolean doPersist = false;
-    if (cname.equals(name)) return;
-
-    SolrCore core = coreContainer.getCore(cname);
-    if (core != null) {
-      doPersist = coreContainer.isPersistent();
-      coreContainer.register(name, core, false);
-      // no core.close() since each entry in the cores map should increase the ref
-    }
-    return;
-  }
-
-
-  /**
    * Handle "UNLOAD" Action
    */
   protected void handleUnloadAction(SolrQueryRequest req,
@@ -852,21 +830,7 @@ public class CoreAdminHandler extends Re
           cname = "";
         }
         try (SolrCore core = coreContainer.getCore(cname)) {
-
           if (core != null) {
-            // try to publish as recovering right away
-            try {
-              coreContainer.getZkController().publish(core.getCoreDescriptor(), ZkStateReader.RECOVERING);
-            }  catch (InterruptedException e) {
-              Thread.currentThread().interrupt();
-              SolrException.log(log, "", e);
-            } catch (Throwable e) {
-              SolrException.log(log, "", e);
-              if (e instanceof Error) {
-                throw (Error) e;
-              }
-            }
-            
             core.getUpdateHandler().getSolrCoreState().doRecovery(coreContainer, core.getCoreDescriptor());
           } else {
             SolrException.log(log, "Could not find core to call recovery:" + cname);

Modified: lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/request/UnInvertedField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/request/UnInvertedField.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/request/UnInvertedField.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/request/UnInvertedField.java Mon Jun 16 19:26:19 2014
@@ -156,7 +156,7 @@ public class UnInvertedField extends Doc
   public long memSize() {
     // can cache the mem size since it shouldn't change
     if (memsz!=0) return memsz;
-    long sz = super.ramUsedInBytes();
+    long sz = super.ramBytesUsed();
     sz += 8*8 + 32; // local fields
     sz += bigTerms.size() * 64;
     for (TopTerm tt : bigTerms.values()) {

Modified: lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/BitDocSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/BitDocSet.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/BitDocSet.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/BitDocSet.java Mon Jun 16 19:26:19 2014
@@ -336,6 +336,11 @@ public class BitDocSet extends DocSetBas
           }
 
           @Override
+          public long ramBytesUsed() {
+            return bs.ramBytesUsed();
+          }
+
+          @Override
           public Bits bits() {
             return new Bits() {
               @Override

Modified: lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/DocSetBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/DocSetBase.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/DocSetBase.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/DocSetBase.java Mon Jun 16 19:26:19 2014
@@ -216,6 +216,11 @@ abstract class DocSetBase implements Doc
           }
 
           @Override
+          public long ramBytesUsed() {
+            return bs.ramBytesUsed();
+          }
+
+          @Override
           public Bits bits() {
             // sparse filters should not use random access
             return null;

Modified: lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java Mon Jun 16 19:26:19 2014
@@ -2465,6 +2465,11 @@ class FilterImpl extends Filter {
     public Bits bits() throws IOException {
       return null;  // don't use random access
     }
+
+    @Override
+    public long ramBytesUsed() {
+      return docIdSet != null ? docIdSet.ramBytesUsed() : 0L;
+    }
   }
 
   private static class FilterIterator extends DocIdSetIterator {

Modified: lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/SortedIntDocSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/SortedIntDocSet.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/SortedIntDocSet.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/SortedIntDocSet.java Mon Jun 16 19:26:19 2014
@@ -25,6 +25,7 @@ import org.apache.lucene.search.DocIdSet
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * <code>SortedIntDocSet</code> represents a sorted set of Lucene Document Ids.
@@ -766,6 +767,11 @@ public class SortedIntDocSet extends Doc
           }
 
           @Override
+          public long ramBytesUsed() {
+            return RamUsageEstimator.sizeOf(docs);
+          }
+          
+          @Override
           public Bits bits() {
             // random access is expensive for this set
             return null;

Modified: lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java Mon Jun 16 19:26:19 2014
@@ -84,6 +84,11 @@ public class ValueSourceRangeFilter exte
        public Bits bits() {
          return null;  // don't use random access
        }
+
+       @Override
+       public long ramBytesUsed() {
+         return 0L;
+       }
      }, acceptDocs);
   }
 

Modified: lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java Mon Jun 16 19:26:19 2014
@@ -437,6 +437,10 @@ public class SolrDispatchFilter extends 
         if (ex instanceof Error) {
           throw (Error) ex;
         }
+        if (ex.getCause() != null && ex.getCause() instanceof Error)  {
+          log.error("An Error was wrapped in another exception - please report complete stacktrace on SOLR-6161", ex);
+          throw (Error) ex.getCause();
+        }
         return;
       } finally {
         try {

Modified: lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/util/SimplePostTool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/util/SimplePostTool.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/util/SimplePostTool.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/java/org/apache/solr/util/SimplePostTool.java Mon Jun 16 19:26:19 2014
@@ -32,6 +32,7 @@ import java.net.MalformedURLException;
 import java.net.ProtocolException;
 import java.net.URL;
 import java.net.URLEncoder;
+import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
@@ -817,10 +818,8 @@ public class SimplePostTool {
         String encoding = DatatypeConverter.printBase64Binary(url.getUserInfo().getBytes(StandardCharsets.US_ASCII));
         urlc.setRequestProperty("Authorization", "Basic " + encoding);
       }
-      if (HttpURLConnection.HTTP_OK != urlc.getResponseCode()) {
-        warn("Solr returned an error #" + urlc.getResponseCode() + 
-            " " + urlc.getResponseMessage() + " for url "+url);
-      }
+      urlc.connect();
+      checkResponseCode(urlc);
     } catch (IOException e) {
       warn("An error occurred posting data to "+url+". Please check that Solr is running.");
     }
@@ -855,45 +854,63 @@ public class SimplePostTool {
           urlc.setRequestProperty("Authorization", "Basic " + encoding);
         }
         if (null != length) urlc.setFixedLengthStreamingMode(length);
-
+        urlc.connect();
       } catch (IOException e) {
         fatal("Connection error (is Solr running at " + solrUrl + " ?): " + e);
         success = false;
       }
       
-      OutputStream out = null;
-      try {
-        out = urlc.getOutputStream();
+      try (final OutputStream out = urlc.getOutputStream()) {
         pipe(data, out);
       } catch (IOException e) {
         fatal("IOException while posting data: " + e);
         success = false;
-      } finally {
-        try { if(out!=null) out.close(); } catch (IOException x) { /*NOOP*/ }
       }
       
-      InputStream in = null;
       try {
-        if (HttpURLConnection.HTTP_OK != urlc.getResponseCode()) {
-          warn("Solr returned an error #" + urlc.getResponseCode() + 
-                " " + urlc.getResponseMessage());
-          success = false;
+        success &= checkResponseCode(urlc);
+        try (final InputStream in = urlc.getInputStream()) {
+          pipe(in, output);
         }
-
-        in = urlc.getInputStream();
-        pipe(in, output);
       } catch (IOException e) {
         warn("IOException while reading response: " + e);
         success = false;
-      } finally {
-        try { if(in!=null) in.close(); } catch (IOException x) { /*NOOP*/ }
       }
-      
     } finally {
-      if(urlc!=null) urlc.disconnect();
+      if (urlc!=null) urlc.disconnect();
     }
     return success;
   }
+  
+  private static boolean checkResponseCode(HttpURLConnection urlc) throws IOException {
+    if (urlc.getResponseCode() >= 400) {
+      warn("Solr returned an error #" + urlc.getResponseCode() + 
+            " (" + urlc.getResponseMessage() + ") for url: " + urlc.getURL());
+      Charset charset = StandardCharsets.ISO_8859_1;
+      final String contentType = urlc.getContentType();
+      // code cloned from ContentStreamBase, but post.jar should be standalone!
+      if (contentType != null) {
+        int idx = contentType.toLowerCase(Locale.ROOT).indexOf("charset=");
+        if (idx > 0) {
+          charset = Charset.forName(contentType.substring(idx + "charset=".length()).trim());
+        }
+      }
+      // Print the response returned by Solr
+      try (InputStream errStream = urlc.getErrorStream()) {
+        if (errStream != null) {
+          BufferedReader br = new BufferedReader(new InputStreamReader(errStream, charset));
+          final StringBuilder response = new StringBuilder("Response: ");
+          int ch;
+          while ((ch = br.read()) != -1) {
+            response.append((char) ch);
+          }
+          warn(response.toString().trim());
+        }
+      }
+      return false;
+    }
+    return true;
+  }
 
   /**
    * Converts a string to an input stream 

Modified: lucene/dev/branches/lucene5752/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml Mon Jun 16 19:26:19 2014
@@ -21,8 +21,8 @@
   <fieldType name="string_simpletext" class="solr.StrField" postingsFormat="SimpleText"/>
   <fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene41"/>
 
-  <fieldType name="string_disk" class="solr.StrField" docValuesFormat="Disk" />
-  <fieldType name="string_memory" class="solr.StrField" docValuesFormat="Lucene49" />
+  <fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene49" />
+  <fieldType name="string_memory" class="solr.StrField" docValuesFormat="Memory" />
 
   <fieldType name="string" class="solr.StrField" />
 

Modified: lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java Mon Jun 16 19:26:19 2014
@@ -266,7 +266,7 @@ public class HttpPartitionTest extends A
   protected void testRf3WithLeaderFailover() throws Exception {
     // now let's create a partition in one of the replicas and outright
     // kill the leader ... see what happens
-    // create a collection that has 1 shard but 2 replicas
+    // create a collection that has 1 shard but 3 replicas
     String testCollectionName = "c8n_1x3_lf"; // _lf is leader fails
     createCollection(testCollectionName, 1, 3, 1);
     cloudClient.setDefaultCollection(testCollectionName);

Modified: lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java Mon Jun 16 19:26:19 2014
@@ -49,10 +49,10 @@ import org.slf4j.LoggerFactory;
 /**
  * Tests a client application's ability to get replication factor
  * information back from the cluster after an add or update.
+ * @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-6157")
  */
 @Slow
 @SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5776")
-@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-6157")
 public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
   
   private static final transient Logger log = 
@@ -132,20 +132,25 @@ public class ReplicationFactorTest exten
    
   @Override
   public void doTest() throws Exception {
+    log.info("replication factor test running");
     waitForThingsToLevelOut(30000);
     
     // test a 1x3 collection
+    log.info("Testing replication factor handling for repfacttest_c8n_1x3");
     testRf3();
 
     // test handling when not using direct updates
+    log.info("Now testing replication factor handling for repfacttest_c8n_2x2");
     testRf2NotUsingDirectUpdates();
+    
+    log.info("replication factor testing complete");
   }
   
   protected void testRf2NotUsingDirectUpdates() throws Exception {
     int numShards = 2;
     int replicationFactor = 2;
     int maxShardsPerNode = 1;
-    String testCollectionName = "c8n_2x2";
+    String testCollectionName = "repfacttest_c8n_2x2";
     String shardId = "shard1";
     int minRf = 2;
     
@@ -184,7 +189,12 @@ public class ReplicationFactorTest exten
     
     // shard1 will have rf=2 but shard2 will only have rf=1
     sendNonDirectUpdateRequestReplica(leader, up, 1, testCollectionName);    
-    sendNonDirectUpdateRequestReplica(replicas.get(0), up, 1, testCollectionName);    
+    sendNonDirectUpdateRequestReplica(replicas.get(0), up, 1, testCollectionName);
+    
+    // heal the partition
+    getProxyForReplica(shard2Replicas.get(0)).reopen();
+    
+    Thread.sleep(2000);
   }
   
   @SuppressWarnings("rawtypes")
@@ -210,7 +220,7 @@ public class ReplicationFactorTest exten
     int numShards = 1;
     int replicationFactor = 3;
     int maxShardsPerNode = 1;
-    String testCollectionName = "c8n_1x3";
+    String testCollectionName = "repfacttest_c8n_1x3";
     String shardId = "shard1";
     int minRf = 2;
     

Modified: lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java Mon Jun 16 19:26:19 2014
@@ -53,9 +53,9 @@ public class TestCodecSupport extends So
     Map<String, SchemaField> fields = h.getCore().getLatestSchema().getFields();
     SchemaField schemaField = fields.get("string_disk_f");
     PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
-    assertEquals("Disk", format.getDocValuesFormatForField(schemaField.getName()).getName());
+    assertEquals("Lucene49", format.getDocValuesFormatForField(schemaField.getName()).getName());
     schemaField = fields.get("string_memory_f");
-    assertEquals("Lucene49",
+    assertEquals("Memory",
         format.getDocValuesFormatForField(schemaField.getName()).getName());
     schemaField = fields.get("string_f");
     assertEquals("Lucene49",
@@ -78,9 +78,9 @@ public class TestCodecSupport extends So
     Codec codec = h.getCore().getCodec();
     PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
 
-    assertEquals("Disk", format.getDocValuesFormatForField("foo_disk").getName());
-    assertEquals("Disk", format.getDocValuesFormatForField("bar_disk").getName());
-    assertEquals("Lucene49", format.getDocValuesFormatForField("foo_memory").getName());
-    assertEquals("Lucene49", format.getDocValuesFormatForField("bar_memory").getName());
+    assertEquals("Lucene49", format.getDocValuesFormatForField("foo_disk").getName());
+    assertEquals("Lucene49", format.getDocValuesFormatForField("bar_disk").getName());
+    assertEquals("Memory", format.getDocValuesFormatForField("foo_memory").getName());
+    assertEquals("Memory", format.getDocValuesFormatForField("bar_memory").getName());
   }
 }

Modified: lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java Mon Jun 16 19:26:19 2014
@@ -36,9 +36,11 @@ import org.apache.solr.util.FileUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.FilenameFilter;
+import java.io.IOException;
 import java.io.InputStream;
 import java.net.URL;
 import java.util.regex.Matcher;
@@ -152,13 +154,9 @@ public class TestReplicationHandlerBacku
 
         File dataDir = new File(master.getDataDir());
 
-        int waitCnt = 0;
         CheckBackupStatus checkBackupStatus = new CheckBackupStatus(firstBackupTimestamp);
         while (true) {
           checkBackupStatus.fetchStatus();
-          if (checkBackupStatus.fail != null) {
-            fail(checkBackupStatus.fail);
-          }
           if (checkBackupStatus.success) {
             if (i == 0) {
               firstBackupTimestamp = checkBackupStatus.backupTimestamp;
@@ -167,10 +165,6 @@ public class TestReplicationHandlerBacku
             break;
           }
           Thread.sleep(200);
-          if (waitCnt == 20) {
-            fail("Backup success not detected:" + checkBackupStatus.response);
-          }
-          waitCnt++;
         }
 
         if (backupThread.fail != null) {
@@ -223,29 +217,20 @@ public class TestReplicationHandlerBacku
       if(!namedBackup) {
         TestUtil.rm(snapDir);
       }
-
     }
   }
 
-  private void testDeleteNamedBackup(String backupNames[]) throws InterruptedException {
+  private void testDeleteNamedBackup(String backupNames[]) throws InterruptedException, IOException {
     for (int i = 0; i < 2; i++) {
       BackupThread deleteBackupThread = new BackupThread(backupNames[i], ReplicationHandler.CMD_DELETE_BACKUP);
       deleteBackupThread.start();
-      int waitCnt = 0;
-      CheckDeleteBackupStatus checkDeleteBackupStatus = new CheckDeleteBackupStatus();
+      CheckDeleteBackupStatus checkDeleteBackupStatus = new CheckDeleteBackupStatus(backupNames[i]);
       while (true) {
-        checkDeleteBackupStatus.fetchStatus();
-        if (checkDeleteBackupStatus.fail != null) {
-          fail(checkDeleteBackupStatus.fail);
-        }
-        if (checkDeleteBackupStatus.success) {
+        boolean success = checkDeleteBackupStatus.fetchStatus();
+        if (success) {
           break;
         }
         Thread.sleep(200);
-        if (waitCnt == 20) {
-          fail("Delete Backup success not detected:" + checkDeleteBackupStatus.response);
-        }
-        waitCnt++;
       }
 
       if (deleteBackupThread.fail != null) {
@@ -255,18 +240,18 @@ public class TestReplicationHandlerBacku
   }
 
   private class CheckBackupStatus {
-    String fail = null;
     String response = null;
     boolean success = false;
     String backupTimestamp = null;
     final String lastBackupTimestamp;
     final Pattern p = Pattern.compile("<str name=\"snapshotCompletedAt\">(.*?)</str>");
+    final Pattern pException = Pattern.compile("<str name=\"snapShootException\">(.*?)</str>");
 
     CheckBackupStatus(String lastBackupTimestamp) {
       this.lastBackupTimestamp = lastBackupTimestamp;
     }
 
-    public void fetchStatus() {
+    public void fetchStatus() throws IOException {
       String masterUrl = buildUrl(masterJetty.getLocalPort(), "/solr") + "/replication?command=" + ReplicationHandler.CMD_DETAILS;
       URL url;
       InputStream stream = null;
@@ -274,6 +259,9 @@ public class TestReplicationHandlerBacku
         url = new URL(masterUrl);
         stream = url.openStream();
         response = IOUtils.toString(stream, "UTF-8");
+        if(pException.matcher(response).find()) {
+          fail("Failed to create backup");
+        }
         if(response.contains("<str name=\"status\">success</str>")) {
           Matcher m = p.matcher(response);
           if(!m.find()) {
@@ -285,8 +273,6 @@ public class TestReplicationHandlerBacku
           }
         }
         stream.close();
-      } catch (Exception e) {
-        fail = e.getMessage();
       } finally {
         IOUtils.closeQuietly(stream);
       }
@@ -338,10 +324,14 @@ public class TestReplicationHandlerBacku
 
   private class CheckDeleteBackupStatus {
     String response = null;
-    boolean success = false;
-    String fail = null;
+    private String backupName;
+    final Pattern p = Pattern.compile("<str name=\"snapshotDeletedAt\">(.*?)</str>");
+
+    private CheckDeleteBackupStatus(String backupName) {
+      this.backupName = backupName;
+    }
 
-    public void fetchStatus() {
+    public boolean fetchStatus() throws IOException {
       String masterUrl = buildUrl(masterJetty.getLocalPort(), context) + "/replication?command=" + ReplicationHandler.CMD_DETAILS;
       URL url;
       InputStream stream = null;
@@ -350,14 +340,18 @@ public class TestReplicationHandlerBacku
         stream = url.openStream();
         response = IOUtils.toString(stream, "UTF-8");
         if(response.contains("<str name=\"status\">success</str>")) {
-          success = true;
+          Matcher m = p.matcher(response);
+          if(m.find()) {
+            return true;
+          }
+        } else if(response.contains("<str name=\"status\">Unable to delete snapshot: " + backupName + "</str>" )) {
+          return false;
         }
         stream.close();
-      } catch (Exception e) {
-        fail = e.getMessage();
       } finally {
         IOUtils.closeQuietly(stream);
       }
+      return false;
     };
   }
 }

Modified: lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/search/AnalyticsMergeStrategyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/search/AnalyticsMergeStrategyTest.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/search/AnalyticsMergeStrategyTest.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/search/AnalyticsMergeStrategyTest.java Mon Jun 16 19:26:19 2014
@@ -83,5 +83,10 @@ public class AnalyticsMergeStrategyTest 
     if(c.intValue() != count) {
       throw new Exception("Count is not correct:"+count+":"+c.intValue());
     }
+
+    long numFound = rsp.getResults().getNumFound();
+    if(c.intValue() != numFound) {
+      throw new Exception("Count does not equal numFound:"+c.intValue()+":"+numFound);
+    }
   }
 }

Modified: lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/search/TestDocSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/search/TestDocSet.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/search/TestDocSet.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/search/TestDocSet.java Mon Jun 16 19:26:19 2014
@@ -33,6 +33,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.MultiReader;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.search.DocIdSet;
@@ -407,6 +408,11 @@ public class TestDocSet extends LuceneTe
       }
       
       @Override
+      public SortedNumericDocValues getSortedNumericDocValues(String field) {
+        return null;
+      }
+      
+      @Override
       public SortedSetDocValues getSortedSetDocValues(String field) {
         return null;
       }

Modified: lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java (original)
+++ lucene/dev/branches/lucene5752/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java Mon Jun 16 19:26:19 2014
@@ -26,7 +26,6 @@ import org.apache.lucene.search.similari
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrDocument;
-import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
@@ -407,16 +406,16 @@ public class DocumentBuilderTest extends
     SolrDocument doc2 = new SolrDocument();
     doc2.addField("foo", randomString);
 
-    assertTrue(assertSolrDocumentEquals(doc1, doc2));
+    assertTrue(compareSolrDocument(doc1, doc2));
 
     doc1.addField("foo", "bar");
 
-    assertFalse(assertSolrDocumentEquals(doc1, doc2));
+    assertFalse(compareSolrDocument(doc1, doc2));
 
     doc1 = new SolrDocument();
     doc1.addField("bar", randomString);
 
-    assertFalse(assertSolrDocumentEquals(doc1, doc2));
+    assertFalse(compareSolrDocument(doc1, doc2));
 
     int randomInt = random().nextInt();
     doc1 = new SolrDocument();
@@ -424,12 +423,12 @@ public class DocumentBuilderTest extends
     doc2 = new SolrDocument();
     doc2.addField("foo", randomInt);
 
-    assertTrue(assertSolrDocumentEquals(doc1, doc2));
+    assertTrue(compareSolrDocument(doc1, doc2));
 
     doc2 = new SolrDocument();
     doc2.addField("bar", randomInt);
 
-    assertFalse(assertSolrDocumentEquals(doc1, doc2));
+    assertFalse(compareSolrDocument(doc1, doc2));
 
   }
 
@@ -442,16 +441,16 @@ public class DocumentBuilderTest extends
     SolrInputDocument doc2 = new SolrInputDocument();
     doc2.addField("foo", randomString);
 
-    assertTrue(assertSolrInputDocumentEquals(doc1, doc2));
+    assertTrue(compareSolrInputDocument(doc1, doc2));
 
     doc1.setDocumentBoost(1.1f);
-    assertFalse(assertSolrInputDocumentEquals(doc1, doc2));
+    assertFalse(compareSolrInputDocument(doc1, doc2));
 
     doc2.setDocumentBoost(1.1f);
-    assertTrue(assertSolrInputDocumentEquals(doc1, doc2));
+    assertTrue(compareSolrInputDocument(doc1, doc2));
 
     doc2.setDocumentBoost(20f);
-    assertFalse(assertSolrInputDocumentEquals(doc1, doc2));
+    assertFalse(compareSolrInputDocument(doc1, doc2));
 
 
     doc1 = new SolrInputDocument();
@@ -463,15 +462,15 @@ public class DocumentBuilderTest extends
     childDoc.addField("foo", "bar");
 
     doc1.addChildDocument(childDoc);
-    assertFalse(assertSolrInputDocumentEquals(doc1, doc2));
+    assertFalse(compareSolrInputDocument(doc1, doc2));
 
     doc2.addChildDocument(childDoc);
-    assertTrue(assertSolrInputDocumentEquals(doc1, doc2));
+    assertTrue(compareSolrInputDocument(doc1, doc2));
 
     SolrInputDocument childDoc1 = new SolrInputDocument();
     childDoc.addField(TestUtil.randomSimpleString(random()), TestUtil.randomSimpleString(random()));
     doc2.addChildDocument(childDoc1);
-    assertFalse(assertSolrInputDocumentEquals(doc1, doc2));
+    assertFalse(compareSolrInputDocument(doc1, doc2));
 
   }
 

Modified: lucene/dev/branches/lucene5752/solr/solrj/src/test/org/apache/solr/common/util/TestJavaBinCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/solrj/src/test/org/apache/solr/common/util/TestJavaBinCodec.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/solrj/src/test/org/apache/solr/common/util/TestJavaBinCodec.java (original)
+++ lucene/dev/branches/lucene5752/solr/solrj/src/test/org/apache/solr/common/util/TestJavaBinCodec.java Mon Jun 16 19:26:19 2014
@@ -32,7 +32,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.io.IOUtils;
-import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.EnumFieldValue;
@@ -183,13 +182,13 @@ public class TestJavaBinCodec extends So
           byte[] b2 = (byte[]) matchObj.get(i);
           assertTrue(Arrays.equals(b1, b2));
         } else if(unmarshaledObj.get(i) instanceof SolrDocument && matchObj.get(i) instanceof SolrDocument ) {
-          assertSolrDocumentEquals(unmarshaledObj.get(i), matchObj.get(i));
+          assertTrue(compareSolrDocument(unmarshaledObj.get(i), matchObj.get(i)));
         } else if(unmarshaledObj.get(i) instanceof SolrDocumentList && matchObj.get(i) instanceof SolrDocumentList ) {
-          assertSolrDocumentEquals(unmarshaledObj.get(i), matchObj.get(i));
+          assertTrue(compareSolrDocumentList(unmarshaledObj.get(i), matchObj.get(i)));
         } else if(unmarshaledObj.get(i) instanceof SolrInputDocument && matchObj.get(i) instanceof SolrInputDocument) {
-          assertSolrInputDocumentEquals(unmarshaledObj.get(i), matchObj.get(i));
+          assertTrue(compareSolrInputDocument(unmarshaledObj.get(i), matchObj.get(i)));
         } else if(unmarshaledObj.get(i) instanceof SolrInputField && matchObj.get(i) instanceof SolrInputField) {
-          assertSolrInputFieldEquals(unmarshaledObj.get(i), matchObj.get(i));
+          assertTrue(assertSolrInputFieldEquals(unmarshaledObj.get(i), matchObj.get(i)));
         } else {
           assertEquals(unmarshaledObj.get(i), matchObj.get(i));
         }
@@ -213,7 +212,7 @@ public class TestJavaBinCodec extends So
       InputStream is = getClass().getResourceAsStream(SOLRJ_JAVABIN_BACKCOMPAT_BIN_CHILD_DOCS);
       SolrDocument sdoc = (SolrDocument) javabin.unmarshal(is);
       SolrDocument matchSolrDoc = generateSolrDocumentWithChildDocs();
-      assertTrue(assertSolrDocumentEquals(sdoc, matchSolrDoc));
+      assertTrue(compareSolrDocument(sdoc, matchSolrDoc));
     } catch (IOException e) {
       throw e;
     }

Modified: lucene/dev/branches/lucene5752/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java (original)
+++ lucene/dev/branches/lucene5752/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java Mon Jun 16 19:26:19 2014
@@ -1873,7 +1873,7 @@ public abstract class SolrTestCaseJ4 ext
     }
   }
 
-  public boolean assertSolrDocumentEquals(Object expected, Object actual) {
+  public boolean compareSolrDocument(Object expected, Object actual) {
 
     if (!(expected instanceof SolrDocument)  || !(actual instanceof SolrDocument)) {
       return false;
@@ -1916,7 +1916,7 @@ public abstract class SolrTestCaseJ4 ext
       Iterator<SolrDocument> childDocsIter1 = solrDocument1.getChildDocuments().iterator();
       Iterator<SolrDocument> childDocsIter2 = solrDocument2.getChildDocuments().iterator();
       while(childDocsIter1.hasNext()) {
-        if(!assertSolrDocumentEquals(childDocsIter1.next(), childDocsIter2.next())) {
+        if(!compareSolrDocument(childDocsIter1.next(), childDocsIter2.next())) {
           return false;
         }
       }
@@ -1924,7 +1924,7 @@ public abstract class SolrTestCaseJ4 ext
     }
   }
 
-  public boolean assertSolrDocumentList(Object expected, Object actual) {
+  public boolean compareSolrDocumentList(Object expected, Object actual) {
     if (!(expected instanceof SolrDocumentList)  || !(actual instanceof SolrDocumentList)) {
       return false;
     }
@@ -1941,14 +1941,14 @@ public abstract class SolrTestCaseJ4 ext
       return false;
     }
     for(int i=0; i<list1.getNumFound(); i++) {
-      if(!assertSolrDocumentEquals(list1.get(i), list2.get(i))) {
+      if(!compareSolrDocument(list1.get(i), list2.get(i))) {
         return false;
       }
     }
     return true;
   }
 
-  public boolean assertSolrInputDocumentEquals(Object expected, Object actual) {
+  public boolean compareSolrInputDocument(Object expected, Object actual) {
 
     if (!(expected instanceof SolrInputDocument) || !(actual instanceof SolrInputDocument)) {
       return false;
@@ -1993,7 +1993,7 @@ public abstract class SolrTestCaseJ4 ext
       Iterator<SolrInputDocument> childDocsIter1 = sdoc1.getChildDocuments().iterator();
       Iterator<SolrInputDocument> childDocsIter2 = sdoc2.getChildDocuments().iterator();
       while(childDocsIter1.hasNext()) {
-        if(!assertSolrInputDocumentEquals(childDocsIter1.next(), childDocsIter2.next())) {
+        if(!compareSolrInputDocument(childDocsIter1.next(), childDocsIter2.next())) {
           return false;
         }
       }