You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ha...@apache.org on 2013/08/30 17:06:49 UTC

svn commit: r1518989 [7/10] - in /lucene/dev/branches/lucene3069: ./ dev-tools/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/ dev-tools/maven/solr/core/src/java/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/...

Modified: lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java Fri Aug 30 15:06:42 2013
@@ -17,21 +17,21 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -52,11 +52,14 @@ import org.apache.lucene.search.ScoreDoc
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 
+import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
+
 /**
  * Abstract class to do basic tests for a docvalues format.
  * NOTE: This test focuses on the docvalues impl, nothing else.
@@ -648,6 +651,43 @@ public abstract class BaseDocValuesForma
     ireader.close();
     directory.close();
   }
+  
+  public void testSortedMergeAwayAllValues() throws IOException {
+    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 SortedDocValuesField("field", new BytesRef("hello")));
+    iwriter.addDocument(doc);
+    iwriter.commit();
+    iwriter.deleteDocuments(new Term("id", "1"));
+    iwriter.forceMerge(1);
+    
+    DirectoryReader ireader = iwriter.getReader();
+    iwriter.close();
+    
+    SortedDocValues dv = getOnlySegmentReader(ireader).getSortedDocValues("field");
+    if (defaultCodecSupportsDocsWithField()) {
+      assertEquals(-1, dv.getOrd(0));
+      assertEquals(0, dv.getValueCount());
+    } else {
+      assertEquals(0, dv.getOrd(0));
+      assertEquals(1, dv.getValueCount());
+      BytesRef ref = new BytesRef();
+      dv.lookupOrd(0, ref);
+      assertEquals(new BytesRef(), ref);
+    }
+    
+    ireader.close();
+    directory.close();
+  }
 
   public void testBytesWithNewline() throws IOException {
     Analyzer analyzer = new MockAnalyzer(random());
@@ -694,7 +734,10 @@ public abstract class BaseDocValuesForma
     BytesRef scratch = new BytesRef();
     dv.lookupOrd(dv.getOrd(0), scratch);
     assertEquals(new BytesRef("hello world 2"), scratch);
-    dv.lookupOrd(dv.getOrd(1), scratch);
+    if (defaultCodecSupportsDocsWithField()) {
+      assertEquals(-1, dv.getOrd(1));
+    }
+    dv.get(1, scratch);
     assertEquals(new BytesRef(""), scratch);
     ireader.close();
     directory.close();
@@ -1049,6 +1092,11 @@ public abstract class BaseDocValuesForma
   public void testRandomSortedBytes() throws IOException {
     Directory dir = newDirectory();
     IndexWriterConfig cfg = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    if (!defaultCodecSupportsDocsWithField()) {
+      // if the codec doesnt support missing, we expect missing to be mapped to byte[]
+      // by the impersonator, but we have to give it a chance to merge them to this
+      cfg.setMergePolicy(newLogMergePolicy());
+    }
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, cfg);
     int numDocs = atLeast(100);
     BytesRefHash hash = new BytesRefHash();
@@ -1073,11 +1121,18 @@ public abstract class BaseDocValuesForma
       doc.add(newTextField("id", "noValue", Field.Store.YES));
       w.addDocument(doc);
     }
-    BytesRef bytesRef = new BytesRef();
-    hash.add(bytesRef); // add empty value for the gaps
+    if (!defaultCodecSupportsDocsWithField()) {
+      BytesRef bytesRef = new BytesRef();
+      hash.add(bytesRef); // add empty value for the gaps
+    }
     if (rarely()) {
       w.commit();
     }
+    if (!defaultCodecSupportsDocsWithField()) {
+      // if the codec doesnt support missing, we expect missing to be mapped to byte[]
+      // by the impersonator, but we have to give it a chance to merge them to this
+      w.forceMerge(1);
+    }
     for (int i = 0; i < numDocs; i++) {
       Document doc = new Document();
       String id = "" + i + numDocs;
@@ -1188,6 +1243,73 @@ public abstract class BaseDocValuesForma
     dir.close();
   }
   
+  private void doTestMissingVsFieldCache(final long minValue, final long maxValue) throws Exception {
+    doTestMissingVsFieldCache(new LongProducer() {
+      @Override
+      long next() {
+        return _TestUtil.nextLong(random(), minValue, maxValue);
+      }
+    });
+  }
+  
+  private void doTestMissingVsFieldCache(LongProducer longs) throws Exception {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+    Field idField = new StringField("id", "", Field.Store.NO);
+    Field indexedField = newStringField("indexed", "", Field.Store.NO);
+    Field dvField = new NumericDocValuesField("dv", 0);
+
+    
+    // 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++) {
+      idField.setStringValue(Integer.toString(i));
+      long value = longs.next();
+      indexedField.setStringValue(Long.toString(value));
+      dvField.setLongValue(value);
+      Document doc = new Document();
+      doc.add(idField);
+      // 1/4 of the time we neglect to add the fields
+      if (random().nextInt(4) > 0) {
+        doc.add(indexedField);
+        doc.add(dvField);
+      }
+      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.close();
+    
+    // compare
+    DirectoryReader ir = DirectoryReader.open(dir);
+    for (AtomicReaderContext context : ir.leaves()) {
+      AtomicReader r = context.reader();
+      Bits expected = FieldCache.DEFAULT.getDocsWithField(r, "indexed");
+      Bits actual = FieldCache.DEFAULT.getDocsWithField(r, "dv");
+      assertEquals(expected, actual);
+    }
+    ir.close();
+    dir.close();
+  }
+  
   public void testBooleanNumericsVsStoredFields() throws Exception {
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
@@ -1202,6 +1324,13 @@ public abstract class BaseDocValuesForma
     }
   }
   
+  public void testByteMissingVsFieldCache() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestMissingVsFieldCache(Byte.MIN_VALUE, Byte.MAX_VALUE);
+    }
+  }
+  
   public void testShortNumericsVsStoredFields() throws Exception {
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
@@ -1209,6 +1338,13 @@ public abstract class BaseDocValuesForma
     }
   }
   
+  public void testShortMissingVsFieldCache() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestMissingVsFieldCache(Short.MIN_VALUE, Short.MAX_VALUE);
+    }
+  }
+  
   public void testIntNumericsVsStoredFields() throws Exception {
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
@@ -1216,6 +1352,13 @@ public abstract class BaseDocValuesForma
     }
   }
   
+  public void testIntMissingVsFieldCache() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestMissingVsFieldCache(Integer.MIN_VALUE, Integer.MAX_VALUE);
+    }
+  }
+  
   public void testLongNumericsVsStoredFields() throws Exception {
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
@@ -1223,6 +1366,13 @@ public abstract class BaseDocValuesForma
     }
   }
   
+  public void testLongMissingVsFieldCache() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestMissingVsFieldCache(Long.MIN_VALUE, Long.MAX_VALUE);
+    }
+  }
+  
   private void doTestBinaryVsStoredFields(int minLength, int maxLength) throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
@@ -1970,6 +2120,13 @@ public abstract class BaseDocValuesForma
       doTestSortedSetVsStoredFields(1, 10);
     }
   }
+
+  private void assertEquals(Bits expected, Bits actual) throws Exception {
+    assertEquals(expected.length(), actual.length());
+    for (int i = 0; i < expected.length(); i++) {
+      assertEquals(expected.get(i), actual.get(i));
+    }
+  }
   
   private void assertEquals(int maxDoc, SortedDocValues expected, SortedDocValues actual) throws Exception {
     assertEquals(maxDoc, new SingletonSortedSetDocValues(expected), new SingletonSortedSetDocValues(actual));
@@ -2197,5 +2354,611 @@ public abstract class BaseDocValuesForma
       doTestNumericsVsStoredFields(longs);
     }
   }
+  
+  public void testTwoNumbersOneMissing() throws IOException {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+    conf.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.YES));
+    doc.add(new NumericDocValuesField("dv1", 0));
+    iw.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    iw.addDocument(doc);
+    iw.forceMerge(1);
+    iw.close();
+    
+    IndexReader ir = DirectoryReader.open(directory);
+    assertEquals(1, ir.leaves().size());
+    AtomicReader ar = ir.leaves().get(0).reader();
+    NumericDocValues dv = ar.getNumericDocValues("dv1");
+    assertEquals(0, dv.get(0));
+    assertEquals(0, dv.get(1));
+    Bits docsWithField = ar.getDocsWithField("dv1");
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+    ir.close();
+    directory.close();
+  }
+  
+  public void testTwoNumbersOneMissingWithMerging() throws IOException {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+    conf.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.YES));
+    doc.add(new NumericDocValuesField("dv1", 0));
+    iw.addDocument(doc);
+    iw.commit();
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    iw.addDocument(doc);
+    iw.forceMerge(1);
+    iw.close();
+    
+    IndexReader ir = DirectoryReader.open(directory);
+    assertEquals(1, ir.leaves().size());
+    AtomicReader ar = ir.leaves().get(0).reader();
+    NumericDocValues dv = ar.getNumericDocValues("dv1");
+    assertEquals(0, dv.get(0));
+    assertEquals(0, dv.get(1));
+    Bits docsWithField = ar.getDocsWithField("dv1");
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+    ir.close();
+    directory.close();
+  }
+  
+  public void testThreeNumbersOneMissingWithMerging() throws IOException {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+    conf.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.YES));
+    doc.add(new NumericDocValuesField("dv1", 0));
+    iw.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    iw.addDocument(doc);
+    iw.commit();
+    doc = new Document();
+    doc.add(new StringField("id", "2", Field.Store.YES));
+    doc.add(new NumericDocValuesField("dv1", 5));
+    iw.addDocument(doc);
+    iw.forceMerge(1);
+    iw.close();
+    
+    IndexReader ir = DirectoryReader.open(directory);
+    assertEquals(1, ir.leaves().size());
+    AtomicReader ar = ir.leaves().get(0).reader();
+    NumericDocValues dv = ar.getNumericDocValues("dv1");
+    assertEquals(0, dv.get(0));
+    assertEquals(0, dv.get(1));
+    assertEquals(5, dv.get(2));
+    Bits docsWithField = ar.getDocsWithField("dv1");
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+    assertTrue(docsWithField.get(2));
+    ir.close();
+    directory.close();
+  }
+  
+  public void testTwoBytesOneMissing() throws IOException {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+    conf.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.YES));
+    doc.add(new BinaryDocValuesField("dv1", new BytesRef()));
+    iw.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    iw.addDocument(doc);
+    iw.forceMerge(1);
+    iw.close();
+    
+    IndexReader ir = DirectoryReader.open(directory);
+    assertEquals(1, ir.leaves().size());
+    AtomicReader ar = ir.leaves().get(0).reader();
+    BinaryDocValues dv = ar.getBinaryDocValues("dv1");
+    BytesRef ref = new BytesRef();
+    dv.get(0, ref);
+    assertEquals(new BytesRef(), ref);
+    dv.get(1, ref);
+    assertEquals(new BytesRef(), ref);
+    Bits docsWithField = ar.getDocsWithField("dv1");
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+    ir.close();
+    directory.close();
+  }
+  
+  public void testTwoBytesOneMissingWithMerging() throws IOException {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+    conf.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.YES));
+    doc.add(new BinaryDocValuesField("dv1", new BytesRef()));
+    iw.addDocument(doc);
+    iw.commit();
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    iw.addDocument(doc);
+    iw.forceMerge(1);
+    iw.close();
+    
+    IndexReader ir = DirectoryReader.open(directory);
+    assertEquals(1, ir.leaves().size());
+    AtomicReader ar = ir.leaves().get(0).reader();
+    BinaryDocValues dv = ar.getBinaryDocValues("dv1");
+    BytesRef ref = new BytesRef();
+    dv.get(0, ref);
+    assertEquals(new BytesRef(), ref);
+    dv.get(1, ref);
+    assertEquals(new BytesRef(), ref);
+    Bits docsWithField = ar.getDocsWithField("dv1");
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+    ir.close();
+    directory.close();
+  }
+  
+  public void testThreeBytesOneMissingWithMerging() throws IOException {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+    conf.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.YES));
+    doc.add(new BinaryDocValuesField("dv1", new BytesRef()));
+    iw.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    iw.addDocument(doc);
+    iw.commit();
+    doc = new Document();
+    doc.add(new StringField("id", "2", Field.Store.YES));
+    doc.add(new BinaryDocValuesField("dv1", new BytesRef("boo")));
+    iw.addDocument(doc);
+    iw.forceMerge(1);
+    iw.close();
+    
+    IndexReader ir = DirectoryReader.open(directory);
+    assertEquals(1, ir.leaves().size());
+    AtomicReader ar = ir.leaves().get(0).reader();
+    BinaryDocValues dv = ar.getBinaryDocValues("dv1");
+    BytesRef ref = new BytesRef();
+    dv.get(0, ref);
+    assertEquals(new BytesRef(), ref);
+    dv.get(1, ref);
+    assertEquals(new BytesRef(), ref);
+    dv.get(2, ref);
+    assertEquals(new BytesRef("boo"), ref);
+    Bits docsWithField = ar.getDocsWithField("dv1");
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+    assertTrue(docsWithField.get(2));
+    ir.close();
+    directory.close();
+  }
+
+  // LUCENE-4853
+  public void testHugeBinaryValues() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    // FSDirectory because SimpleText will consume gobbs of
+    // space when storing big binary values:
+    Directory d = newFSDirectory(_TestUtil.getTempDir("hugeBinaryValues"));
+    boolean doFixed = random().nextBoolean();
+    int numDocs;
+    int fixedLength = 0;
+    if (doFixed) {
+      // Sometimes make all values fixed length since some
+      // codecs have different code paths for this:
+      numDocs = _TestUtil.nextInt(random(), 10, 20);
+      fixedLength = _TestUtil.nextInt(random(), 65537, 256*1024);
+    } else {
+      numDocs = _TestUtil.nextInt(random(), 100, 200);
+    }
+    IndexWriter w = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer));
+    List<byte[]> docBytes = new ArrayList<byte[]>();
+    long totalBytes = 0;
+    for(int docID=0;docID<numDocs;docID++) {
+      // we don't use RandomIndexWriter because it might add
+      // more docvalues than we expect !!!!
+
+      // Must be > 64KB in size to ensure more than 2 pages in
+      // PagedBytes would be needed:
+      int numBytes;
+      if (doFixed) {
+        numBytes = fixedLength;
+      } else if (docID == 0 || random().nextInt(5) == 3) {
+        numBytes = _TestUtil.nextInt(random(), 65537, 3*1024*1024);
+      } else {
+        numBytes = _TestUtil.nextInt(random(), 1, 1024*1024);
+      }
+      totalBytes += numBytes;
+      if (totalBytes > 5 * 1024*1024) {
+        break;
+      }
+      byte[] bytes = new byte[numBytes];
+      random().nextBytes(bytes);
+      docBytes.add(bytes);
+      Document doc = new Document();      
+      BytesRef b = new BytesRef(bytes);
+      b.length = bytes.length;
+      doc.add(new BinaryDocValuesField("field", b));
+      doc.add(new StringField("id", ""+docID, Field.Store.YES));
+      try {
+        w.addDocument(doc);
+      } catch (IllegalArgumentException iae) {
+        if (iae.getMessage().indexOf("is too large") == -1) {
+          throw iae;
+        } else {
+          // OK: some codecs can't handle binary DV > 32K
+          assertFalse(codecAcceptsHugeBinaryValues("field"));
+          w.rollback();
+          d.close();
+          return;
+        }
+      }
+    }
+    
+    DirectoryReader r;
+    try {
+      r = w.getReader();
+    } catch (IllegalArgumentException iae) {
+      if (iae.getMessage().indexOf("is too large") == -1) {
+        throw iae;
+      } else {
+        assertFalse(codecAcceptsHugeBinaryValues("field"));
+
+        // OK: some codecs can't handle binary DV > 32K
+        w.rollback();
+        d.close();
+        return;
+      }
+    }
+    w.close();
 
+    AtomicReader ar = SlowCompositeReaderWrapper.wrap(r);
+
+    BinaryDocValues s = FieldCache.DEFAULT.getTerms(ar, "field");
+    for(int docID=0;docID<docBytes.size();docID++) {
+      StoredDocument doc = ar.document(docID);
+      BytesRef bytes = new BytesRef();
+      s.get(docID, bytes);
+      byte[] expected = docBytes.get(Integer.parseInt(doc.get("id")));
+      assertEquals(expected.length, bytes.length);
+      assertEquals(new BytesRef(expected), bytes);
+    }
+
+    assertTrue(codecAcceptsHugeBinaryValues("field"));
+
+    ar.close();
+    d.close();
+  }
+
+  // TODO: get this out of here and into the deprecated codecs (4.0, 4.2)
+  public void testHugeBinaryValueLimit() throws Exception {
+    // We only test DVFormats that have a limit
+    assumeFalse("test requires codec with limits on max binary field length", codecAcceptsHugeBinaryValues("field"));
+    Analyzer analyzer = new MockAnalyzer(random());
+    // FSDirectory because SimpleText will consume gobbs of
+    // space when storing big binary values:
+    Directory d = newFSDirectory(_TestUtil.getTempDir("hugeBinaryValues"));
+    boolean doFixed = random().nextBoolean();
+    int numDocs;
+    int fixedLength = 0;
+    if (doFixed) {
+      // Sometimes make all values fixed length since some
+      // codecs have different code paths for this:
+      numDocs = _TestUtil.nextInt(random(), 10, 20);
+      fixedLength = Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH;
+    } else {
+      numDocs = _TestUtil.nextInt(random(), 100, 200);
+    }
+    IndexWriter w = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer));
+    List<byte[]> docBytes = new ArrayList<byte[]>();
+    long totalBytes = 0;
+    for(int docID=0;docID<numDocs;docID++) {
+      // we don't use RandomIndexWriter because it might add
+      // more docvalues than we expect !!!!
+
+      // Must be > 64KB in size to ensure more than 2 pages in
+      // PagedBytes would be needed:
+      int numBytes;
+      if (doFixed) {
+        numBytes = fixedLength;
+      } else if (docID == 0 || random().nextInt(5) == 3) {
+        numBytes = Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH;
+      } else {
+        numBytes = _TestUtil.nextInt(random(), 1, Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH);
+      }
+      totalBytes += numBytes;
+      if (totalBytes > 5 * 1024*1024) {
+        break;
+      }
+      byte[] bytes = new byte[numBytes];
+      random().nextBytes(bytes);
+      docBytes.add(bytes);
+      Document doc = new Document();      
+      BytesRef b = new BytesRef(bytes);
+      b.length = bytes.length;
+      doc.add(new BinaryDocValuesField("field", b));
+      doc.add(new StringField("id", ""+docID, Field.Store.YES));
+      w.addDocument(doc);
+    }
+    
+    DirectoryReader r = w.getReader();
+    w.close();
+
+    AtomicReader ar = SlowCompositeReaderWrapper.wrap(r);
+
+    BinaryDocValues s = FieldCache.DEFAULT.getTerms(ar, "field");
+    for(int docID=0;docID<docBytes.size();docID++) {
+      StoredDocument doc = ar.document(docID);
+      BytesRef bytes = new BytesRef();
+      s.get(docID, bytes);
+      byte[] expected = docBytes.get(Integer.parseInt(doc.get("id")));
+      assertEquals(expected.length, bytes.length);
+      assertEquals(new BytesRef(expected), bytes);
+    }
+
+    ar.close();
+    d.close();
+  }
+  
+  /** Tests dv against stored fields with threads (binary/numeric/sorted, no missing) */
+  public void testThreads() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+    Document doc = new Document();
+    Field idField = new StringField("id", "", Field.Store.NO);
+    Field storedBinField = new StoredField("storedBin", new byte[0]);
+    Field dvBinField = new BinaryDocValuesField("dvBin", new BytesRef());
+    Field dvSortedField = new SortedDocValuesField("dvSorted", new BytesRef());
+    Field storedNumericField = new StoredField("storedNum", "");
+    Field dvNumericField = new NumericDocValuesField("dvNum", 0);
+    doc.add(idField);
+    doc.add(storedBinField);
+    doc.add(dvBinField);
+    doc.add(dvSortedField);
+    doc.add(storedNumericField);
+    doc.add(dvNumericField);
+    
+    // index some docs
+    int numDocs = atLeast(300);
+    for (int i = 0; i < numDocs; i++) {
+      idField.setStringValue(Integer.toString(i));
+      int length = _TestUtil.nextInt(random(), 0, 8);
+      byte buffer[] = new byte[length];
+      random().nextBytes(buffer);
+      storedBinField.setBytesValue(buffer);
+      dvBinField.setBytesValue(buffer);
+      dvSortedField.setBytesValue(buffer);
+      long numericValue = random().nextLong();
+      storedNumericField.setStringValue(Long.toString(numericValue));
+      dvNumericField.setLongValue(numericValue);
+      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)));
+    }
+    writer.close();
+    
+    // compare
+    final DirectoryReader ir = DirectoryReader.open(dir);
+    int numThreads = _TestUtil.nextInt(random(), 2, 7);
+    Thread threads[] = new Thread[numThreads];
+    final CountDownLatch startingGun = new CountDownLatch(1);
+    
+    for (int i = 0; i < threads.length; i++) {
+      threads[i] = new Thread() {
+        @Override
+        public void run() {
+          try {
+            startingGun.await();
+            for (AtomicReaderContext context : ir.leaves()) {
+              AtomicReader r = context.reader();
+              BinaryDocValues binaries = r.getBinaryDocValues("dvBin");
+              SortedDocValues sorted = r.getSortedDocValues("dvSorted");
+              NumericDocValues numerics = r.getNumericDocValues("dvNum");
+              for (int j = 0; j < r.maxDoc(); j++) {
+                BytesRef binaryValue = r.document(j).getBinaryValue("storedBin");
+                BytesRef scratch = new BytesRef();
+                binaries.get(j, scratch);
+                assertEquals(binaryValue, scratch);
+                sorted.get(j, scratch);
+                assertEquals(binaryValue, scratch);
+                String expected = r.document(j).get("storedNum");
+                assertEquals(Long.parseLong(expected), numerics.get(j));
+              }
+            }
+            _TestUtil.checkReader(ir);
+          } catch (Exception e) {
+            throw new RuntimeException(e);
+          }
+        }
+      };
+      threads[i].start();
+    }
+    startingGun.countDown();
+    for (Thread t : threads) {
+      t.join();
+    }
+    ir.close();
+    dir.close();
+  }
+  
+  /** Tests dv against stored fields with threads (all types + missing) */
+  public void testThreads2() throws Exception {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    assumeTrue("Codec does not support SORTED_SET", defaultCodecSupportsSortedSet());
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+    Field idField = new StringField("id", "", Field.Store.NO);
+    Field storedBinField = new StoredField("storedBin", new byte[0]);
+    Field dvBinField = new BinaryDocValuesField("dvBin", new BytesRef());
+    Field dvSortedField = new SortedDocValuesField("dvSorted", new BytesRef());
+    Field storedNumericField = new StoredField("storedNum", "");
+    Field dvNumericField = new NumericDocValuesField("dvNum", 0);
+    
+    // index some docs
+    int numDocs = atLeast(300);
+    for (int i = 0; i < numDocs; i++) {
+      idField.setStringValue(Integer.toString(i));
+      int length = _TestUtil.nextInt(random(), 0, 8);
+      byte buffer[] = new byte[length];
+      random().nextBytes(buffer);
+      storedBinField.setBytesValue(buffer);
+      dvBinField.setBytesValue(buffer);
+      dvSortedField.setBytesValue(buffer);
+      long numericValue = random().nextLong();
+      storedNumericField.setStringValue(Long.toString(numericValue));
+      dvNumericField.setLongValue(numericValue);
+      Document doc = new Document();
+      doc.add(idField);
+      if (random().nextInt(4) > 0) {
+        doc.add(storedBinField);
+        doc.add(dvBinField);
+        doc.add(dvSortedField);
+      }
+      if (random().nextInt(4) > 0) {
+        doc.add(storedNumericField);
+        doc.add(dvNumericField);
+      }
+      int numSortedSetFields = random().nextInt(3);
+      Set<String> values = new TreeSet<String>();
+      for (int j = 0; j < numSortedSetFields; j++) {
+        values.add(_TestUtil.randomSimpleString(random()));
+      }
+      for (String v : values) {
+        doc.add(new SortedSetDocValuesField("dvSortedSet", new BytesRef(v)));
+        doc.add(new StoredField("storedSortedSet", v));
+      }
+      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)));
+    }
+    writer.close();
+    
+    // compare
+    final DirectoryReader ir = DirectoryReader.open(dir);
+    int numThreads = _TestUtil.nextInt(random(), 2, 7);
+    Thread threads[] = new Thread[numThreads];
+    final CountDownLatch startingGun = new CountDownLatch(1);
+    
+    for (int i = 0; i < threads.length; i++) {
+      threads[i] = new Thread() {
+        @Override
+        public void run() {
+          try {
+            startingGun.await();
+            for (AtomicReaderContext context : ir.leaves()) {
+              AtomicReader r = context.reader();
+              BinaryDocValues binaries = r.getBinaryDocValues("dvBin");
+              Bits binaryBits = r.getDocsWithField("dvBin");
+              SortedDocValues sorted = r.getSortedDocValues("dvSorted");
+              Bits sortedBits = r.getDocsWithField("dvSorted");
+              NumericDocValues numerics = r.getNumericDocValues("dvNum");
+              Bits numericBits = r.getDocsWithField("dvNum");
+              SortedSetDocValues sortedSet = r.getSortedSetDocValues("dvSortedSet");
+              Bits sortedSetBits = r.getDocsWithField("dvSortedSet");
+              for (int j = 0; j < r.maxDoc(); j++) {
+                BytesRef binaryValue = r.document(j).getBinaryValue("storedBin");
+                if (binaryValue != null) {
+                  if (binaries != null) {
+                    BytesRef scratch = new BytesRef();
+                    binaries.get(j, scratch);
+                    assertEquals(binaryValue, scratch);
+                    sorted.get(j, scratch);
+                    assertEquals(binaryValue, scratch);
+                    assertTrue(binaryBits.get(j));
+                    assertTrue(sortedBits.get(j));
+                  }
+                } else if (binaries != null) {
+                  assertFalse(binaryBits.get(j));
+                  assertFalse(sortedBits.get(j));
+                  assertEquals(-1, sorted.getOrd(j));
+                }
+               
+                String number = r.document(j).get("storedNum");
+                if (number != null) {
+                  if (numerics != null) {
+                    assertEquals(Long.parseLong(number), numerics.get(j));
+                  }
+                } else if (numerics != null) {
+                  assertFalse(numericBits.get(j));
+                  assertEquals(0, numerics.get(j));
+                }
+                
+                String values[] = r.document(j).getValues("storedSortedSet");
+                if (values.length > 0) {
+                  assertNotNull(sortedSet);
+                  sortedSet.setDocument(j);
+                  for (int i = 0; i < values.length; i++) {
+                    long ord = sortedSet.nextOrd();
+                    assertTrue(ord != SortedSetDocValues.NO_MORE_ORDS);
+                    BytesRef value = new BytesRef();
+                    sortedSet.lookupOrd(ord, value);
+                    assertEquals(values[i], value.utf8ToString());
+                  }
+                  assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSet.nextOrd());
+                  assertTrue(sortedSetBits.get(j));
+                } else if (sortedSet != null) {
+                  sortedSet.setDocument(j);
+                  assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSet.nextOrd());
+                  assertFalse(sortedSetBits.get(j));
+                }
+              }
+            }
+            _TestUtil.checkReader(ir);
+          } catch (Exception e) {
+            throw new RuntimeException(e);
+          }
+        }
+      };
+      threads[i].start();
+    }
+    startingGun.countDown();
+    for (Thread t : threads) {
+      t.join();
+    }
+    ir.close();
+    dir.close();
+  }
+
+  protected boolean codecAcceptsHugeBinaryValues(String field) {
+    return true;
+  }
 }

Modified: lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java Fri Aug 30 15:06:42 2013
@@ -32,7 +32,7 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.compressing.CompressingCodec;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene45.Lucene45Codec;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoubleField;
@@ -502,7 +502,7 @@ public abstract class BaseStoredFieldsFo
     // get another codec, other than the default: so we are merging segments across different codecs
     final Codec otherCodec;
     if ("SimpleText".equals(Codec.getDefault().getName())) {
-      otherCodec = new Lucene42Codec();
+      otherCodec = new Lucene45Codec();
     } else {
       otherCodec = new SimpleTextCodec();
     }

Modified: lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java Fri Aug 30 15:06:42 2013
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Set;
 
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.FilterIterator;
 
 /**
@@ -137,6 +138,11 @@ public final class FieldFilterAtomicRead
   }
 
   @Override
+  public Bits getDocsWithField(String field) throws IOException {
+    return hasField(field) ? super.getDocsWithField(field) : null;
+  }
+
+  @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder("FieldFilterAtomicReader(reader=");
     sb.append(in).append(", fields=");

Modified: lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java Fri Aug 30 15:06:42 2013
@@ -35,12 +35,12 @@ import org.apache.lucene.codecs.lucene41
 import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
 import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
 import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
-import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
+import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat;
 import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
-import org.apache.lucene.codecs.cheapbastard.CheapBastardDocValuesFormat;
 import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
 import org.apache.lucene.codecs.memory.DirectPostingsFormat;
+import org.apache.lucene.codecs.memory.MemoryDocValuesFormat;
 import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
 import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
 import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
@@ -63,7 +63,7 @@ import org.apache.lucene.util._TestUtil;
  * documents in different orders and the test will still be deterministic
  * and reproducable.
  */
-public class RandomCodec extends Lucene42Codec {
+public class RandomCodec extends Lucene45Codec {
   /** Shuffled list of postings formats to use for new mappings */
   private List<PostingsFormat> formats = new ArrayList<PostingsFormat>();
   
@@ -158,11 +158,11 @@ public class RandomCodec extends Lucene4
         new TempPulsing41PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock));
 
     addDocValues(avoidCodecs,
-        new Lucene42DocValuesFormat(),
+        new Lucene45DocValuesFormat(),
         new DiskDocValuesFormat(),
+        new MemoryDocValuesFormat(),
         new SimpleTextDocValuesFormat(),
-        new AssertingDocValuesFormat(),
-        new CheapBastardDocValuesFormat());
+        new AssertingDocValuesFormat());
 
     Collections.shuffle(formats, random);
     Collections.shuffle(dvFormats, random);

Modified: lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java Fri Aug 30 15:06:42 2013
@@ -340,7 +340,7 @@ public class RandomIndexWriter implement
     w.forceMerge(maxSegmentCount);
   }
   
-  private static final class TestPointInfoStream extends InfoStream {
+  static final class TestPointInfoStream extends InfoStream {
     private final InfoStream delegate;
     private final TestPoint testPoint;
     

Modified: lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java Fri Aug 30 15:06:42 2013
@@ -25,6 +25,7 @@ import junit.framework.Assert;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.index.AllDeletedFilterReader;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
@@ -38,7 +39,6 @@ import org.apache.lucene.store.MockDirec
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util._TestUtil;
 
 import static org.apache.lucene.util.LuceneTestCase.TEST_VERSION_CURRENT;
 
@@ -193,7 +193,7 @@ public class QueryUtils {
   static final IndexReader[] emptyReaders = new IndexReader[8];
   static {
     try {
-      emptyReaders[0] = makeEmptyIndex(new Random(0), 0);
+      emptyReaders[0] = new MultiReader();
       emptyReaders[4] = makeEmptyIndex(new Random(0), 4);
       emptyReaders[5] = makeEmptyIndex(new Random(0), 5);
       emptyReaders[7] = makeEmptyIndex(new Random(0), 7);
@@ -202,31 +202,18 @@ public class QueryUtils {
     }
   }
 
-  private static DirectoryReader makeEmptyIndex(Random random, final int numDeletedDocs) 
-    throws IOException {
+  private static IndexReader makeEmptyIndex(Random random, final int numDocs) throws IOException {
+    assert numDocs > 0;
     Directory d = new MockDirectoryWrapper(random, new RAMDirectory());
-      IndexWriter w = new IndexWriter(d, new IndexWriterConfig(
-        TEST_VERSION_CURRENT, new MockAnalyzer(random)));
-      for (int i = 0; i < numDeletedDocs; i++) {
-        w.addDocument(new Document());
-      }
-      w.commit();
-      w.deleteDocuments( new MatchAllDocsQuery() );
-      _TestUtil.keepFullyDeletedSegments(w);
-      w.commit();
-
-      if (0 < numDeletedDocs)
-        Assert.assertTrue("writer has no deletions", w.hasDeletions());
-
-      Assert.assertEquals("writer is missing some deleted docs", 
-                          numDeletedDocs, w.maxDoc());
-      Assert.assertEquals("writer has non-deleted docs", 
-                          0, w.numDocs());
-      w.close();
-      DirectoryReader r = DirectoryReader.open(d);
-      Assert.assertEquals("reader has wrong number of deleted docs", 
-                          numDeletedDocs, r.numDeletedDocs());
-      return r;
+    IndexWriter w = new IndexWriter(d, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
+    for (int i = 0; i < numDocs; i++) {
+      w.addDocument(new Document());
+    }
+    w.forceMerge(1);
+    w.commit();
+    w.close();
+    DirectoryReader reader = DirectoryReader.open(d);
+    return new AllDeletedFilterReader(LuceneTestCase.getOnlySegmentReader(reader));
   }
 
   /** alternate scorer skipTo(),skipTo(),next(),next(),skipTo(),skipTo(), etc

Modified: lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Fri Aug 30 15:06:42 2013
@@ -1372,6 +1372,16 @@ public abstract class LuceneTestCase ext
     }
     return true;
   }
+  
+  /** Returns true if the codec "supports" docsWithField 
+   * (other codecs return MatchAllBits, because you couldnt write missing values before) */
+  public static boolean defaultCodecSupportsDocsWithField() {
+    String name = Codec.getDefault().getName();
+    if (name.equals("Lucene40") || name.equals("Lucene41") || name.equals("Lucene42")) {
+      return false;
+    }
+    return true;
+  }
 
   public void assertReaderEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
     assertReaderStatisticsEquals(info, leftReader, rightReader);
@@ -1966,6 +1976,20 @@ public abstract class LuceneTestCase ext
           assertNull(info, rightValues);
         }
       }
+      
+      {
+        Bits leftBits = MultiDocValues.getDocsWithField(leftReader, field);
+        Bits rightBits = MultiDocValues.getDocsWithField(rightReader, field);
+        if (leftBits != null && rightBits != null) {
+          assertEquals(info, leftBits.length(), rightBits.length());
+          for (int i = 0; i < leftBits.length(); i++) {
+            assertEquals(info, leftBits.get(i), rightBits.get(i));
+          }
+        } else {
+          assertNull(info, leftBits);
+          assertNull(info, rightBits);
+        }
+      }
     }
   }
   

Modified: lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java Fri Aug 30 15:06:42 2013
@@ -41,6 +41,8 @@ import org.apache.lucene.codecs.lucene40
 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.lucene42.Lucene42RWCodec;
+import org.apache.lucene.codecs.lucene45.Lucene45Codec;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.index.RandomCodec;
 import org.apache.lucene.search.RandomSimilarityProvider;
@@ -162,6 +164,13 @@ final class TestRuleSetupAndRestoreClass
                                                  !shouldAvoidCodec("Lucene41"))) { 
       codec = Codec.forName("Lucene41");
       assert codec instanceof Lucene41RWCodec : "fix your classpath to have tests-framework.jar before lucene-core.jar";
+    } else if ("Lucene42".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
+                                                 "random".equals(TEST_POSTINGSFORMAT) &&
+                                                 "random".equals(TEST_DOCVALUESFORMAT) &&
+                                                  randomVal == 2 &&
+                                                  !shouldAvoidCodec("Lucene42"))) { 
+      codec = Codec.forName("Lucene42");
+      assert codec instanceof Lucene42RWCodec : "fix your classpath to have tests-framework.jar before lucene-core.jar";
     } else if (("random".equals(TEST_POSTINGSFORMAT) == false) || ("random".equals(TEST_DOCVALUESFORMAT) == false)) {
       // the user wired postings or DV: this is messy
       // refactor into RandomCodec....
@@ -185,7 +194,7 @@ final class TestRuleSetupAndRestoreClass
         dvFormat = DocValuesFormat.forName(TEST_DOCVALUESFORMAT);
       }
       
-      codec = new Lucene42Codec() {       
+      codec = new Lucene45Codec() {       
         @Override
         public PostingsFormat getPostingsFormatForField(String field) {
           return format;

Modified: lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java Fri Aug 30 15:06:42 2013
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.PrintStream;
-import java.lang.reflect.Method;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.nio.CharBuffer;
@@ -45,7 +44,8 @@ import java.util.zip.ZipFile;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
@@ -59,12 +59,12 @@ import org.apache.lucene.document.Numeri
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.CheckIndex;
 import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
 import org.apache.lucene.index.CheckIndex.Status.FieldNormStatus;
 import org.apache.lucene.index.CheckIndex.Status.StoredFieldStatus;
 import org.apache.lucene.index.CheckIndex.Status.TermIndexStatus;
 import org.apache.lucene.index.CheckIndex.Status.TermVectorStatus;
+import org.apache.lucene.index.CheckIndex;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
@@ -703,7 +703,7 @@ public class _TestUtil {
     if (LuceneTestCase.VERBOSE) {
       System.out.println("forcing postings format to:" + format);
     }
-    return new Lucene42Codec() {
+    return new Lucene45Codec() {
       @Override
       public PostingsFormat getPostingsFormatForField(String field) {
         return format;
@@ -721,7 +721,7 @@ public class _TestUtil {
     if (LuceneTestCase.VERBOSE) {
       System.out.println("forcing docvalues format to:" + format);
     }
-    return new Lucene42Codec() {
+    return new Lucene45Codec() {
       @Override
       public DocValuesFormat getDocValuesFormatForField(String field) {
         return format;
@@ -744,6 +744,28 @@ public class _TestUtil {
     }
   }
 
+  public static String getDocValuesFormat(String field) {
+    return getDocValuesFormat(Codec.getDefault(), field);
+  }
+  
+  public static String getDocValuesFormat(Codec codec, String field) {
+    DocValuesFormat f = codec.docValuesFormat();
+    if (f instanceof PerFieldDocValuesFormat) {
+      return ((PerFieldDocValuesFormat) f).getDocValuesFormatForField(field).getName();
+    } else {
+      return f.getName();
+    }
+  }
+
+  // TODO: remove this, push this test to Lucene40/Lucene42 codec tests
+  public static boolean fieldSupportsHugeBinaryDocValues(String field) {
+    String dvFormat = getDocValuesFormat(field);
+    if (dvFormat.equals("Lucene40") || dvFormat.equals("Lucene42") || dvFormat.equals("Memory")) {
+      return false;
+    }
+    return true;
+  }
+
   public static boolean anyFilesExceptWriteLock(Directory dir) throws IOException {
     String[] files = dir.listAll();
     if (files.length > 1 || (files.length == 1 && !files[0].equals("write.lock"))) {
@@ -788,19 +810,6 @@ public class _TestUtil {
     });
     Assert.assertEquals("Reflection does not produce same map", reflectedValues, map);
   }
-
-  public static void keepFullyDeletedSegments(IndexWriter w) {
-    try {
-      // Carefully invoke what is a package-private (test
-      // only, internal) method on IndexWriter:
-      Method m = IndexWriter.class.getDeclaredMethod("keepFullyDeletedSegments");
-      m.setAccessible(true);
-      m.invoke(w);
-    } catch (Exception e) {
-      // Should not happen?
-      throw new RuntimeException(e);
-    }
-  }
   
   /** 
    * insecure, fast version of File.createTempFile

Modified: lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Fri Aug 30 15:06:42 2013
@@ -21,3 +21,4 @@ org.apache.lucene.codecs.compressing.Hig
 org.apache.lucene.codecs.compressing.dummy.DummyCompressingCodec
 org.apache.lucene.codecs.lucene40.Lucene40RWCodec
 org.apache.lucene.codecs.lucene41.Lucene41RWCodec
+org.apache.lucene.codecs.lucene42.Lucene42RWCodec

Modified: lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat Fri Aug 30 15:06:42 2013
@@ -14,4 +14,4 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.asserting.AssertingDocValuesFormat
-org.apache.lucene.codecs.cheapbastard.CheapBastardDocValuesFormat
\ No newline at end of file
+org.apache.lucene.codecs.lucene42.Lucene42RWDocValuesFormat

Modified: lucene/dev/branches/lucene3069/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/CHANGES.txt?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/CHANGES.txt (original)
+++ lucene/dev/branches/lucene3069/solr/CHANGES.txt Fri Aug 30 15:06:42 2013
@@ -33,8 +33,13 @@ Apache ZooKeeper 3.4.5
 Upgrading from Solr 4.x
 ----------------------
 
-The "file" attribute of infoStream in solrconfig.xml is removed. Control this 
-via your logging configuration (org.apache.solr.update.LoggingInfoStream) instead.
+* The "file" attribute of infoStream in solrconfig.xml is removed. Control this 
+  via your logging configuration (org.apache.solr.update.LoggingInfoStream) instead.
+
+* UniqFieldsUpdateProcessorFactory no longer supports the <lst named="fields"> init
+  param style that was deprecated in Solr 4.5.  If you are still using this syntax, 
+  update your configs to use <arr name="fieldName"> instead. See SOLR-4249 for more 
+  details.
 
 Detailed Change List
 ----------------------
@@ -66,9 +71,15 @@ Upgrading from Solr 4.4.0
   would be chosen arbitrarily and silently.  Starting with 4.5, configuration 
   parsing will fail with an error in situations like this.  If you see error 
   messages such as "solrconfig.xml contains more than one value for config path: 
-  indexConfig/infoStream" check your solrconfig.xml file for multiple occurrences 
-  of "infoStream" and delete the one that you do not wish to use.  See SOLR-4953
-  for more details.
+  XXXXX" or "Found Z configuration sections when at most 1 is allowed matching 
+  expression: XXXXX" check your solrconfig.xml file for multiple occurrences of 
+  XXXXX and delete the ones that you do not wish to use.  See SOLR-4953 & 
+  SOLR-5108 for more details.
+
+* The UniqFieldsUpdateProcessorFactory has been improved to support all of the
+  FieldMutatingUpdateProcessorFactory selector options.  The <lst named="fields"> 
+  init param option is now deprecated and should be replaced with the more standard
+  <arr name="fieldName">.  See SOLR-4249 for more details.
 
 Detailed Change List
 ----------------------
@@ -91,7 +102,21 @@ New Features
 * SOLR-4808: Persist and use router,replicationFactor and maxShardsPerNode at Collection and Shard level (Noble Paul, Shalin Mangar)
 * SOLR-5006: CREATESHARD command for 'implicit' shards (Noble Paul)
 * SOLR-5017: Allow sharding based on the value of a field (Noble Paul)
-* SOLR-4222:create custom sharded collection via collections API (Noble Paul)
+* SOLR-4222: create custom sharded collection via collections API (Noble Paul)
+* SOLR-4718: Allow solr.xml to be stored in ZooKeeper
+* SOLR-5156: Enhance ZkCLI to allow uploading of arbitrary files to ZK.
+
+* SOLR-5165: Single-valued docValues fields no longer require a default value.
+  Additionally they work with sortMissingFirst, sortMissingLast, facet.missing, 
+  exists() in function queries, etc.  (Robert Muir)
+
+* SOLR-5182: Add NoOpRegenerator, a regenerator for custom per-segment caches
+  where items are preserved across commits.  (Robert Muir)
+
+* SOLR-4249: UniqFieldsUpdateProcessorFactory now extends 
+  FieldMutatingUpdateProcessorFactory and supports all of it's selector options. Use
+  of the "fields" init param is now deprecated in favor of "fieldName" (hossman)
+  
 
 Bug Fixes
 ----------------------
@@ -116,6 +141,37 @@ Bug Fixes
   
 * SOLR-5119: Managed schema problems after adding fields via Schema Rest API.
   (Nils Kübler, Steve Rowe)
+  
+* SOLR-5133: HdfsUpdateLog can fail to close a FileSystem instance if init 
+  is called more than once. (Mark Miller)
+
+* SOLR-5135: Harden Collection API deletion of /collections/$collection 
+  ZooKeeper node. (Mark Miller)
+
+* SOLR-4764: When using NRT, just init the first reader from IndexWriter.
+  (Robert Muir, Mark Miller)
+  
+* SOLR-5122: Fixed bug in spellcheck.collateMaxCollectDocs.  Eliminates risk 
+  of divide by zero, and makes estimated hit counts meaningful in non-optimized
+  indexes.  (hossman)
+  
+* SOLR-3936: Fixed QueryElevationComponent sorting when used with Grouping
+  (Michael Garski via hossman)
+
+* SOLR-5171: SOLR Admin gui works in IE9, breaks in IE10. (Joseph L Howard via
+  steffkes)
+
+* SOLR-5174: Admin UI - Query View doesn't highlight (json) Result if it 
+  contains HTML Tags (steffkes)
+  
+* SOLR-4817 Solr should not fall back to the back compat built in solr.xml in SolrCloud 
+  mode (Erick Erickson)
+
+* SOLR-5112: Show full message in Admin UI Logging View (Matthew Keeney via
+  steffkes)
+
+* SOLR-5190: SolrEntityProcessor substitutes variables only once in child entities
+  (Harsh Chawla, shalin)
 
 Optimizations
 ----------------------
@@ -123,6 +179,11 @@ Optimizations
 * SOLR-5044: Admin UI - Note on Core-Admin about directories while creating 
   core (steffkes)
 
+* SOLR-5134: Have HdfsIndexOutput extend BufferedIndexOutput. 
+  (Mark Miller, Uwe Schindler)
+
+ * SOLR-5057: QueryResultCache should not related with the order of fq's list (Feihong Huang via Erick Erickson)
+
 Other Changes
 ----------------------
 
@@ -130,19 +191,23 @@ Other Changes
   The solr.clustering.enabled system property is set to 'true' by default.
   (ehatcher, Dawid Weiss)
 
-* SOLR-4914: Factor out core list persistence and discovery into a
-  new CoresLocator interface. (Alan Woodward)
+* SOLR-4914, SOLR-5162: Factor out core list persistence and discovery into a
+  new CoresLocator interface. (Alan Woodward, Shawn Heisey)
 
 * SOLR-5056: Improve type safety of ConfigSolr class. (Alan Woodward)
 
 * SOLR-4951: Better randomization of MergePolicy in Solr tests (hossman)
 
-* SOLR-4953: Make XML Configuration parsing fail if an xpath matches multiple 
-  nodes when only a single value is expected.  (hossman)
+* SOLR-4953, SOLR-5108: Make XML Configuration parsing fail if an xpath matches 
+  multiple nodes when only a single value or plugin instance is expected.  
+  (hossman)
 
 * The routing parameter "shard.keys" is deprecated as part of SOLR-5017 .The new parameter name is '_route_' .
   The old parameter should continue to work for another release  (Noble Paul)
 
+* SOLR-5173: Solr-core's Maven configuration includes test-only Hadoop
+  dependencies as indirect compile-time dependencies.
+  (Chris Collins, Steve Rowe)
 
 ==================  4.4.0 ==================
 

Modified: lucene/dev/branches/lucene3069/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java (original)
+++ lucene/dev/branches/lucene3069/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java Fri Aug 30 15:06:42 2013
@@ -71,8 +71,6 @@ public class SolrEntityProcessor extends
   private String[] fields;
   private String requestHandler;// 'qt' param
   private int timeout = TIMEOUT_SECS;
-  
-  private boolean initDone = false;
 
   /**
    * Factory method that returns a {@link HttpClient} instance used for interfacing with a source Solr service.
@@ -122,28 +120,22 @@ public class SolrEntityProcessor extends
    * external synchronization. 
    */
   private void buildIterator() {
-    if (rowIterator == null) {
-      // We could use an AtomicBoolean but there's no need since this method
-      // would require anyway external synchronization
-      if (!initDone) {
-        initDone = true;
-        SolrDocumentList solrDocumentList = doQuery(0);
+    if (rowIterator != null)  {
+      SolrDocumentListIterator documentListIterator = (SolrDocumentListIterator) rowIterator;
+      if (!documentListIterator.hasNext() && documentListIterator.hasMoreRows()) {
+        SolrDocumentList solrDocumentList = doQuery(documentListIterator
+            .getStart() + documentListIterator.getSize());
         if (solrDocumentList != null) {
           rowIterator = new SolrDocumentListIterator(solrDocumentList);
         }
       }
-      return;
-    }
-    
-    SolrDocumentListIterator documentListIterator = (SolrDocumentListIterator) rowIterator;
-    if (!documentListIterator.hasNext() && documentListIterator.hasMoreRows()) {
-      SolrDocumentList solrDocumentList = doQuery(documentListIterator
-          .getStart() + documentListIterator.getSize());
+    } else  {
+      SolrDocumentList solrDocumentList = doQuery(0);
       if (solrDocumentList != null) {
         rowIterator = new SolrDocumentListIterator(solrDocumentList);
       }
+      return;
     }
-    
   }
   
   protected SolrDocumentList doQuery(int start) {

Modified: lucene/dev/branches/lucene3069/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java (original)
+++ lucene/dev/branches/lucene3069/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java Fri Aug 30 15:06:42 2013
@@ -41,6 +41,7 @@ import java.util.List;
  */
 public class TestContentStreamDataSource extends AbstractDataImportHandlerTestCase {
   private static final String CONF_DIR = "dih/solr/collection1/conf/";
+  private static final String ROOT_DIR = "dih/solr/";
   SolrInstance instance = null;
   JettySolrRunner jetty;
 
@@ -144,6 +145,11 @@ public class TestContentStreamDataSource
       return CONF_DIR + "contentstream-solrconfig.xml";
     }
 
+    public String getSolrXmlFile() {
+      return ROOT_DIR + "solr.xml";
+    }
+
+
     public void setUp() throws Exception {
 
       File home = new File(TEMP_DIR,
@@ -158,6 +164,7 @@ public class TestContentStreamDataSource
       dataDir.mkdirs();
       confDir.mkdirs();
 
+      FileUtils.copyFile(getFile(getSolrXmlFile()), new File(homeDir, "solr.xml"));
       File f = new File(confDir, "solrconfig.xml");
       FileUtils.copyFile(getFile(getSolrConfigFile()), f);
       f = new File(confDir, "schema.xml");

Modified: lucene/dev/branches/lucene3069/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java (original)
+++ lucene/dev/branches/lucene3069/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java Fri Aug 30 15:06:42 2013
@@ -49,7 +49,8 @@ public class TestSolrEntityProcessorEndT
   private static final String SOLR_CONFIG = "dataimport-solrconfig.xml";
   private static final String SOLR_SCHEMA = "dataimport-schema.xml";
   private static final String SOURCE_CONF_DIR = "dih" + File.separator + "solr" + File.separator + "collection1" + File.separator + "conf" + File.separator;
-  
+  private static final String ROOT_DIR = "dih" + File.separator + "solr" + File.separator;
+
   private static final String DEAD_SOLR_SERVER = "http://[ff01::114]:33332/solr";
   
   private static final List<Map<String,Object>> DB_DOCS = new ArrayList<Map<String,Object>>();
@@ -61,7 +62,7 @@ public class TestSolrEntityProcessorEndT
     dbDoc.put("dbid_s", "1");
     dbDoc.put("dbdesc_s", "DbDescription");
     DB_DOCS.add(dbDoc);
-    
+
     Map<String,Object> solrDoc = new HashMap<String,Object>();
     solrDoc.put("id", "1");
     solrDoc.put("desc", "SolrDescription");
@@ -201,8 +202,19 @@ public class TestSolrEntityProcessorEndT
     assertQ(req("*:*"), "//result[@numFound='0']");
     
     try {
-      MockDataSource.setIterator("select * from x", DB_DOCS.iterator());
-      addDocumentsToSolr(SOLR_DOCS);
+      List<Map<String,Object>> DOCS = new ArrayList<Map<String,Object>>(DB_DOCS);
+      Map<String, Object> doc = new HashMap<String, Object>();
+      doc.put("dbid_s", "2");
+      doc.put("dbdesc_s", "DbDescription2");
+      DOCS.add(doc);
+      MockDataSource.setIterator("select * from x", DOCS.iterator());
+
+      DOCS = new ArrayList<Map<String,Object>>(SOLR_DOCS);
+      Map<String,Object> solrDoc = new HashMap<String,Object>();
+      solrDoc.put("id", "2");
+      solrDoc.put("desc", "SolrDescription2");
+      DOCS.add(solrDoc);
+      addDocumentsToSolr(DOCS);
       runFullImport(getDihConfigTagsInnerEntity());
     } catch (Exception e) {
       LOG.error(e.getMessage(), e);
@@ -211,12 +223,15 @@ public class TestSolrEntityProcessorEndT
       MockDataSource.clearCache();
     }
     
-    assertQ(req("*:*"), "//result[@numFound='1']");
+    assertQ(req("*:*"), "//result[@numFound='2']");
     assertQ(req("id:1"), "//result/doc/str[@name='id'][.='1']",
         "//result/doc/str[@name='dbdesc_s'][.='DbDescription']",
         "//result/doc/str[@name='dbid_s'][.='1']",
         "//result/doc/arr[@name='desc'][.='SolrDescription']");
-    
+    assertQ(req("id:2"), "//result/doc/str[@name='id'][.='2']",
+        "//result/doc/str[@name='dbdesc_s'][.='DbDescription2']",
+        "//result/doc/str[@name='dbid_s'][.='2']",
+        "//result/doc/arr[@name='desc'][.='SolrDescription2']");
   }
   
   public void testFullImportWrongSolrUrl() {
@@ -293,7 +308,11 @@ public class TestSolrEntityProcessorEndT
     public String getSolrConfigFile() {
       return SOURCE_CONF_DIR + "dataimport-solrconfig.xml";
     }
-    
+
+    public String getSolrXmlFile() {
+      return ROOT_DIR + "solr.xml";
+    }
+
     public void setUp() throws Exception {
       
       File home = new File(TEMP_DIR, getClass().getName() + "-"
@@ -306,7 +325,8 @@ public class TestSolrEntityProcessorEndT
       homeDir.mkdirs();
       dataDir.mkdirs();
       confDir.mkdirs();
-      
+
+      FileUtils.copyFile(getFile(getSolrXmlFile()), new File(homeDir, "solr.xml"));
       File f = new File(confDir, "solrconfig.xml");
       FileUtils.copyFile(getFile(getSolrConfigFile()), f);
       f = new File(confDir, "schema.xml");

Modified: lucene/dev/branches/lucene3069/solr/contrib/uima/src/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/contrib/uima/src/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessor.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/contrib/uima/src/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessor.java (original)
+++ lucene/dev/branches/lucene3069/solr/contrib/uima/src/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessor.java Fri Aug 30 15:06:42 2013
@@ -20,7 +20,6 @@ package org.apache.solr.uima.processor;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.uima.processor.SolrUIMAConfiguration.MapField;
 import org.apache.lucene.analysis.uima.ae.AEProvider;
@@ -48,19 +47,16 @@ public class UIMAUpdateRequestProcessor 
   SolrUIMAConfiguration solrUIMAConfiguration;
 
   private AEProvider aeProvider;
-  
-  private SolrCore solrCore;
 
-  public UIMAUpdateRequestProcessor(UpdateRequestProcessor next, SolrCore solrCore,
+  public UIMAUpdateRequestProcessor(UpdateRequestProcessor next, String coreName,
       SolrUIMAConfiguration config) {
     super(next);
-    initialize(solrCore, config);
+    initialize(coreName, config);
   }
 
-  private void initialize(SolrCore solrCore, SolrUIMAConfiguration config) {
-    this.solrCore = solrCore;
+  private void initialize(String coreName, SolrUIMAConfiguration config) {
     solrUIMAConfiguration = config;
-    aeProvider = AEProviderFactory.getInstance().getAEProvider(solrCore.getName(),
+    aeProvider = AEProviderFactory.getInstance().getAEProvider(coreName,
             solrUIMAConfiguration.getAePath(), solrUIMAConfiguration.getRuntimeParameters());
   }
 

Modified: lucene/dev/branches/lucene3069/solr/contrib/uima/src/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/contrib/uima/src/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessorFactory.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/contrib/uima/src/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessorFactory.java (original)
+++ lucene/dev/branches/lucene3069/solr/contrib/uima/src/java/org/apache/solr/uima/processor/UIMAUpdateRequestProcessorFactory.java Fri Aug 30 15:06:42 2013
@@ -41,7 +41,7 @@ public class UIMAUpdateRequestProcessorF
   @Override
   public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp,
           UpdateRequestProcessor next) {
-    return new UIMAUpdateRequestProcessor(next, req.getCore(),
+    return new UIMAUpdateRequestProcessor(next, req.getCore().getName(),
             new SolrUIMAConfigurationReader(args).readSolrUIMAConfiguration());
   }
 

Modified: lucene/dev/branches/lucene3069/solr/core/ivy.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/ivy.xml?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/ivy.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/ivy.xml Fri Aug 30 15:06:42 2013
@@ -25,9 +25,9 @@
   <configurations>
     <!-- artifacts in the "compile" and "compile.hadoop" configurations will go into solr/core/lib/ -->
     <conf name="compile" transitive="false"/>
-    <conf name="test" transitive="false"/>
     <conf name="compile.hadoop" transitive="false"/>
     <!-- artifacts in the "test" and "test.DfsMiniCluster" configuration will go into solr/core/test-lib/ -->
+    <conf name="test" transitive="false"/>
     <conf name="test.DfsMiniCluster" transitive="false"/>
   </configurations>
 
@@ -49,6 +49,11 @@
     <dependency org="org.objenesis" name="objenesis" rev="1.2" conf="test->*"/>
 
     <dependency org="org.apache.hadoop" name="hadoop-common" rev="&hadoop.version;" conf="compile.hadoop->*"/>
+    <!--
+      hadoop-hdfs, hadoop-annotations and hadoop-auth are runtime dependencies,
+      so even though they are not compile-time dependencies, they are included
+      here as such so that they are included in the runtime distribution.
+     -->
     <dependency org="org.apache.hadoop" name="hadoop-hdfs" rev="&hadoop.version;" conf="compile.hadoop->*"/>
     <dependency org="org.apache.hadoop" name="hadoop-annotations" rev="&hadoop.version;" conf="compile.hadoop->*"/>
     <dependency org="org.apache.hadoop" name="hadoop-auth" rev="&hadoop.version;" conf="compile.hadoop->*"/>

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java Fri Aug 30 15:06:42 2013
@@ -26,7 +26,6 @@ import java.util.Random;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicLong;
-
 import java.net.URL;
 import java.net.MalformedURLException;
 
@@ -499,6 +498,10 @@ public class JettySolrRunner {
   public void setCoreNodeName(String coreNodeName) {
     this.coreNodeName = coreNodeName;
   }
+
+  public String getSolrHome() {
+    return solrHome;
+  }
 }
 
 class NoLog implements Logger {

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/cloud/Overseer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/cloud/Overseer.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/cloud/Overseer.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/cloud/Overseer.java Fri Aug 30 15:06:42 2013
@@ -17,8 +17,17 @@ package org.apache.solr.cloud;
  * the License.
  */
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ClosableThread;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
@@ -36,16 +45,6 @@ import org.apache.zookeeper.KeeperExcept
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
 /**
  * Cluster leader. Responsible node assignments, cluster state file?
  */
@@ -295,17 +294,6 @@ public class Overseer {
         final String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
         assert collection.length() > 0 : message;
         
-        try {
-          if (!zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection, true)) {
-            log.warn("Could not find collection node for " + collection + ", skipping publish state");
-          }
-        } catch (KeeperException e) {
-          throw new SolrException(ErrorCode.SERVER_ERROR, e);
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new SolrException(ErrorCode.SERVER_ERROR, e);
-        }
-        
         String coreNodeName = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP);
         if (coreNodeName == null) {
           coreNodeName = getAssignedCoreNodeName(state, message);

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java Fri Aug 30 15:06:42 2013
@@ -232,31 +232,57 @@ public class OverseerCollectionProcessor
     return new OverseerSolrResponse(results);
   }
 
-  private void deleteCollection(ZkNodeProps message, NamedList results) throws KeeperException, InterruptedException {
-    ModifiableSolrParams params = new ModifiableSolrParams();
-    params.set(CoreAdminParams.ACTION, CoreAdminAction.UNLOAD.toString());
-    params.set(CoreAdminParams.DELETE_INSTANCE_DIR, true);
-    params.set(CoreAdminParams.DELETE_DATA_DIR, true);
-    collectionCmd(zkStateReader.getClusterState(), message, params, results, null);
-
-    ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
-        Overseer.REMOVECOLLECTION, "name", message.getStr("name"));
-    Overseer.getInQueue(zkStateReader.getZkClient()).offer(ZkStateReader.toJSON(m));
-
-    // wait for a while until we don't see the collection
-    long now = System.currentTimeMillis();
-    long timeout = now + 30000;
-    boolean removed = false;
-    while (System.currentTimeMillis() < timeout) {
-      Thread.sleep(100);
-      removed = !zkStateReader.getClusterState().getCollections().contains(message.getStr("name"));
-      if (removed) {
-        Thread.sleep(100); // just a bit of time so it's more likely other readers see on return
-        break;
+  private void deleteCollection(ZkNodeProps message, NamedList results)
+      throws KeeperException, InterruptedException {
+    String collection = message.getStr("name");
+    try {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set(CoreAdminParams.ACTION, CoreAdminAction.UNLOAD.toString());
+      params.set(CoreAdminParams.DELETE_INSTANCE_DIR, true);
+      params.set(CoreAdminParams.DELETE_DATA_DIR, true);
+      collectionCmd(zkStateReader.getClusterState(), message, params, results,
+          null);
+      
+      ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
+          Overseer.REMOVECOLLECTION, "name", collection);
+      Overseer.getInQueue(zkStateReader.getZkClient()).offer(
+          ZkStateReader.toJSON(m));
+      
+      // wait for a while until we don't see the collection
+      long now = System.currentTimeMillis();
+      long timeout = now + 30000;
+      boolean removed = false;
+      while (System.currentTimeMillis() < timeout) {
+        Thread.sleep(100);
+        removed = !zkStateReader.getClusterState().getCollections()
+            .contains(message.getStr("name"));
+        if (removed) {
+          Thread.sleep(100); // just a bit of time so it's more likely other
+                             // readers see on return
+          break;
+        }
+      }
+      if (!removed) {
+        throw new SolrException(ErrorCode.SERVER_ERROR,
+            "Could not fully remove collection: " + message.getStr("name"));
+      }
+      
+    } finally {
+      
+      try {
+        if (zkStateReader.getZkClient().exists(
+            ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection, true)) {
+          zkStateReader.getZkClient().clean(
+              ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection);
+        }
+      } catch (InterruptedException e) {
+        SolrException.log(log, "Cleaning up collection in zk was interrupted:"
+            + collection, e);
+        Thread.currentThread().interrupt();
+      } catch (KeeperException e) {
+        SolrException.log(log, "Problem cleaning up collection in zk:"
+            + collection, e);
       }
-    }
-    if (!removed) {
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Could not fully remove collection: " + message.getStr("name"));
     }
   }
 
@@ -992,11 +1018,6 @@ public class OverseerCollectionProcessor
     
     DocCollection coll = clusterState.getCollection(collectionName);
     
-    if (coll == null) {
-      throw new SolrException(ErrorCode.BAD_REQUEST,
-          "Could not find collection:" + collectionName);
-    }
-    
     for (Map.Entry<String,Slice> entry : coll.getSlicesMap().entrySet()) {
       Slice slice = entry.getValue();
       sliceCmd(clusterState, params, stateMatcher, slice);

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java Fri Aug 30 15:06:42 2013
@@ -3,6 +3,7 @@ package org.apache.solr.cloud;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
@@ -19,7 +20,9 @@ import org.xml.sax.SAXException;
 
 import javax.xml.parsers.ParserConfigurationException;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.util.List;
 import java.util.concurrent.TimeoutException;
 
@@ -44,6 +47,7 @@ public class ZkCLI {
   
   private static final String MAKEPATH = "makepath";
   private static final String PUT = "put";
+  private static final String PUT_FILE = "putfile";
   private static final String DOWNCONFIG = "downconfig";
   private static final String ZK_CLI_NAME = "ZkCLI";
   private static final String HELP = "help";
@@ -87,7 +91,8 @@ public class ZkCLI {
         .hasArg(true)
         .withDescription(
             "cmd to run: " + BOOTSTRAP + ", " + UPCONFIG + ", " + DOWNCONFIG
-                + ", " + LINKCONFIG + ", " + MAKEPATH + ", "+ PUT + ", "+ LIST + ", " + CLEAR).create(CMD));
+                + ", " + LINKCONFIG + ", " + MAKEPATH + ", " + PUT + ", " + PUT_FILE + ","
+                + LIST + ", " + CLEAR).create(CMD));
 
     Option zkHostOption = new Option("z", ZKHOST, true,
         "ZooKeeper host address");
@@ -131,6 +136,7 @@ public class ZkCLI {
         System.out.println("zkcli.sh -zkhost localhost:9983 -cmd " + LINKCONFIG + " -" + COLLECTION + " collection1" + " -" + CONFNAME + " myconf");
         System.out.println("zkcli.sh -zkhost localhost:9983 -cmd " + MAKEPATH + " /apache/solr");
         System.out.println("zkcli.sh -zkhost localhost:9983 -cmd " + PUT + " /solr.conf 'conf data'");
+        System.out.println("zkcli.sh -zkhost localhost:9983 -cmd " + PUT_FILE + " /solr.xml /User/myuser/solr/solr.xml");
         System.out.println("zkcli.sh -zkhost localhost:9983 -cmd " + CLEAR + " /solr");
         System.out.println("zkcli.sh -zkhost localhost:9983 -cmd " + LIST);
         return;
@@ -244,6 +250,20 @@ public class ZkCLI {
           }
           zkClient.create(arglist.get(0).toString(), arglist.get(1).toString().getBytes("UTF-8"),
                           acl, CreateMode.PERSISTENT, true);
+        } else if (line.getOptionValue(CMD).equals(PUT_FILE)) {
+          List arglist = line.getArgList();
+          if (arglist.size() != 2) {
+            System.out.println("-" + PUT_FILE + " requires two args - the path to create in ZK and the path to the local file");
+            System.exit(1);
+          }
+          InputStream is = new FileInputStream(arglist.get(1).toString());
+          try {
+            zkClient.create(arglist.get(0).toString(), IOUtils.toByteArray(is),
+                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, true);
+          } finally {
+            IOUtils.closeQuietly(is);
+          }
+
         }
       } finally {
         if (solrPort != null) {