You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2013/07/04 01:26:45 UTC

svn commit: r1499601 [9/20] - in /lucene/dev/branches/security: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/solr/core/src/test/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/analysis/stempel/ dev-to...

Modified: lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/TestFieldCacheSanityChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/TestFieldCacheSanityChecker.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/TestFieldCacheSanityChecker.java (original)
+++ lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/TestFieldCacheSanityChecker.java Wed Jul  3 23:26:32 2013
@@ -16,21 +16,25 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleField;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.document.FloatField;
+import org.apache.lucene.document.IntField;
+import org.apache.lucene.document.LongField;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.MultiReader;
 import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.MultiReader;
 import org.apache.lucene.index.SlowCompositeReaderWrapper;
+import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.FieldCacheSanityChecker.Insanity;
 import org.apache.lucene.util.FieldCacheSanityChecker.InsanityType;
 
-import java.io.IOException;
-
 public class TestFieldCacheSanityChecker extends LuceneTestCase {
 
   protected AtomicReader readerA;
@@ -51,18 +55,14 @@ public class TestFieldCacheSanityChecker
 
     long theLong = Long.MAX_VALUE;
     double theDouble = Double.MAX_VALUE;
-    byte theByte = Byte.MAX_VALUE;
-    short theShort = Short.MAX_VALUE;
     int theInt = Integer.MAX_VALUE;
     float theFloat = Float.MAX_VALUE;
     for (int i = 0; i < NUM_DOCS; i++){
       Document doc = new Document();
-      doc.add(newStringField("theLong", String.valueOf(theLong--), Field.Store.NO));
-      doc.add(newStringField("theDouble", String.valueOf(theDouble--), Field.Store.NO));
-      doc.add(newStringField("theByte", String.valueOf(theByte--), Field.Store.NO));
-      doc.add(newStringField("theShort", String.valueOf(theShort--), Field.Store.NO));
-      doc.add(newStringField("theInt", String.valueOf(theInt--), Field.Store.NO));
-      doc.add(newStringField("theFloat", String.valueOf(theFloat--), Field.Store.NO));
+      doc.add(new LongField("theLong", theLong--, Field.Store.NO));
+      doc.add(new DoubleField("theDouble", theDouble--, Field.Store.NO));
+      doc.add(new IntField("theInt", theInt--, Field.Store.NO));
+      doc.add(new FloatField("theFloat", theFloat--, Field.Store.NO));
       if (0 == i % 3) {
         wA.addDocument(doc);
       } else {
@@ -95,12 +95,12 @@ public class TestFieldCacheSanityChecker
     cache.purgeAllCaches();
 
     cache.getDoubles(readerA, "theDouble", false);
-    cache.getDoubles(readerA, "theDouble", FieldCache.DEFAULT_DOUBLE_PARSER, false);
-    cache.getDoubles(readerAclone, "theDouble", FieldCache.DEFAULT_DOUBLE_PARSER, false);
-    cache.getDoubles(readerB, "theDouble", FieldCache.DEFAULT_DOUBLE_PARSER, false);
+    cache.getDoubles(readerA, "theDouble", FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, false);
+    cache.getDoubles(readerAclone, "theDouble", FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, false);
+    cache.getDoubles(readerB, "theDouble", FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, false);
 
     cache.getInts(readerX, "theInt", false);
-    cache.getInts(readerX, "theInt", FieldCache.DEFAULT_INT_PARSER, false);
+    cache.getInts(readerX, "theInt", FieldCache.NUMERIC_UTILS_INT_PARSER, false);
 
     // // // 
 
@@ -119,9 +119,8 @@ public class TestFieldCacheSanityChecker
     FieldCache cache = FieldCache.DEFAULT;
     cache.purgeAllCaches();
 
-    cache.getInts(readerX, "theInt", FieldCache.DEFAULT_INT_PARSER, false);
+    cache.getInts(readerX, "theInt", FieldCache.NUMERIC_UTILS_INT_PARSER, false);
     cache.getTerms(readerX, "theInt");
-    cache.getBytes(readerX, "theByte", false);
 
     // // // 
 
@@ -147,8 +146,6 @@ public class TestFieldCacheSanityChecker
     cache.getTerms(readerB, "theInt");
     cache.getTerms(readerX, "theInt");
 
-    cache.getBytes(readerX, "theByte", false);
-
 
     // // // 
 

Modified: lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java (original)
+++ lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java Wed Jul  3 23:26:32 2013
@@ -34,7 +34,7 @@ import org.apache.lucene.util.packed.Pac
 import org.junit.Ignore;
 import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
 
-@Ignore("Requires tons of heap to run (10G works)")
+@Ignore("Requires tons of heap to run (420G works)")
 @TimeoutSuite(millis = 100 * TimeUnits.HOUR)
 public class Test2BFST extends LuceneTestCase {
 
@@ -50,12 +50,12 @@ public class Test2BFST extends LuceneTes
     for(int doPackIter=0;doPackIter<2;doPackIter++) {
       boolean doPack = doPackIter == 1;
 
-      // Build FST w/ NoOutputs and stop when nodeCount > 3B
+      // Build FST w/ NoOutputs and stop when nodeCount > 2.2B
       if (!doPack) {
         System.out.println("\nTEST: 3B nodes; doPack=false output=NO_OUTPUTS");
         Outputs<Object> outputs = NoOutputs.getSingleton();
         Object NO_OUTPUT = outputs.getNoOutput();
-        final Builder<Object> b = new Builder<Object>(FST.INPUT_TYPE.BYTE1, 0, 0, false, false, Integer.MAX_VALUE, outputs,
+        final Builder<Object> b = new Builder<Object>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs,
                                                       null, doPack, PackedInts.COMPACT, true, 15);
 
         int count = 0;
@@ -72,7 +72,7 @@ public class Test2BFST extends LuceneTes
           if (count % 100000 == 0) {
             System.out.println(count + ": " + b.fstSizeInBytes() + " bytes; " + b.getTotStateCount() + " nodes");
           }
-          if (b.getTotStateCount() > LIMIT) {
+          if (b.getTotStateCount() > Integer.MAX_VALUE + 100L * 1024 * 1024) {
             break;
           }
           nextInput(r, ints2);

Modified: lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Wed Jul  3 23:26:32 2013
@@ -126,7 +126,7 @@ public class TestFSTs extends LuceneTest
 
       // FST ord pos int
       {
-        final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+        final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
         final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms2.length);
         for(int idx=0;idx<terms2.length;idx++) {
           pairs.add(new FSTTester.InputOutput<Long>(terms2[idx], (long) idx));
@@ -171,7 +171,7 @@ public class TestFSTs extends LuceneTest
 
     // PositiveIntOutput (ord)
     {
-      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
       final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms.length);
       for(int idx=0;idx<terms.length;idx++) {
         pairs.add(new FSTTester.InputOutput<Long>(terms[idx], (long) idx));
@@ -181,8 +181,7 @@ public class TestFSTs extends LuceneTest
 
     // PositiveIntOutput (random monotonically increasing positive number)
     {
-      final boolean doShare = random().nextBoolean();
-      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(doShare);
+      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
       final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms.length);
       long lastOutput = 0;
       for(int idx=0;idx<terms.length;idx++) {
@@ -190,12 +189,12 @@ public class TestFSTs extends LuceneTest
         lastOutput = value;
         pairs.add(new FSTTester.InputOutput<Long>(terms[idx], value));
       }
-      new FSTTester<Long>(random(), dir, inputMode, pairs, outputs, doShare).doTest(true);
+      new FSTTester<Long>(random(), dir, inputMode, pairs, outputs, true).doTest(true);
     }
 
     // PositiveIntOutput (random positive number)
     {
-      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(random().nextBoolean());
+      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
       final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms.length);
       for(int idx=0;idx<terms.length;idx++) {
         pairs.add(new FSTTester.InputOutput<Long>(terms[idx], _TestUtil.nextLong(random(), 0, Long.MAX_VALUE)));
@@ -205,8 +204,8 @@ public class TestFSTs extends LuceneTest
 
     // Pair<ord, (random monotonically increasing positive number>
     {
-      final PositiveIntOutputs o1 = PositiveIntOutputs.getSingleton(random().nextBoolean());
-      final PositiveIntOutputs o2 = PositiveIntOutputs.getSingleton(random().nextBoolean());
+      final PositiveIntOutputs o1 = PositiveIntOutputs.getSingleton();
+      final PositiveIntOutputs o2 = PositiveIntOutputs.getSingleton();
       final PairOutputs<Long,Long> outputs = new PairOutputs<Long,Long>(o1, o2);
       final List<FSTTester.InputOutput<PairOutputs.Pair<Long,Long>>> pairs = new ArrayList<FSTTester.InputOutput<PairOutputs.Pair<Long,Long>>>(terms.length);
       long lastOutput = 0;
@@ -306,7 +305,7 @@ public class TestFSTs extends LuceneTest
     }
     IndexReader r = DirectoryReader.open(writer, true);
     writer.close();
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(random().nextBoolean());
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
 
     final boolean doRewrite = random().nextBoolean();
 
@@ -653,8 +652,8 @@ public class TestFSTs extends LuceneTest
 
     if (storeOrds && storeDocFreqs) {
       // Store both ord & docFreq:
-      final PositiveIntOutputs o1 = PositiveIntOutputs.getSingleton(true);
-      final PositiveIntOutputs o2 = PositiveIntOutputs.getSingleton(false);
+      final PositiveIntOutputs o1 = PositiveIntOutputs.getSingleton();
+      final PositiveIntOutputs o2 = PositiveIntOutputs.getSingleton();
       final PairOutputs<Long,Long> outputs = new PairOutputs<Long,Long>(o1, o2);
       new VisitTerms<PairOutputs.Pair<Long,Long>>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
         Random rand;
@@ -669,7 +668,7 @@ public class TestFSTs extends LuceneTest
       }.run(limit, verify, false);
     } else if (storeOrds) {
       // Store only ords
-      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
       new VisitTerms<Long>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
         @Override
         public Long getOutput(IntsRef input, int ord) {
@@ -678,7 +677,7 @@ public class TestFSTs extends LuceneTest
       }.run(limit, verify, true);
     } else if (storeDocFreqs) {
       // Store only docFreq
-      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(false);
+      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
       new VisitTerms<Long>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
         Random rand;
         @Override
@@ -781,7 +780,7 @@ public class TestFSTs extends LuceneTest
     // smaller FST if the outputs grow monotonically.  But
     // if numbers are "random", false should give smaller
     // final size:
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
 
     // Build an FST mapping BytesRef -> Long
     final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
@@ -1100,7 +1099,7 @@ public class TestFSTs extends LuceneTest
   }
 
   public void testFinalOutputOnEndState() throws Exception {
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
 
     final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean(), PackedInts.DEFAULT, true, 15);
     builder.add(Util.toUTF32("stat", new IntsRef()), 17L);
@@ -1115,7 +1114,7 @@ public class TestFSTs extends LuceneTest
   }
 
   public void testInternalFinalState() throws Exception {
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     final boolean willRewrite = random().nextBoolean();
     final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite, PackedInts.DEFAULT, true, 15);
     builder.add(Util.toIntsRef(new BytesRef("stat"), new IntsRef()), outputs.getNoOutput());
@@ -1136,7 +1135,7 @@ public class TestFSTs extends LuceneTest
   // Make sure raw FST can differentiate between final vs
   // non-final end nodes
   public void testNonFinalStopNode() throws Exception {
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     final Long nothing = outputs.getNoOutput();
     final Builder<Long> b = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
 
@@ -1216,7 +1215,7 @@ public class TestFSTs extends LuceneTest
   };
 
   public void testShortestPaths() throws Exception {
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
 
     final IntsRef scratch = new IntsRef();
@@ -1258,8 +1257,8 @@ public class TestFSTs extends LuceneTest
   public void testShortestPathsWFST() throws Exception {
 
     PairOutputs<Long,Long> outputs = new PairOutputs<Long,Long>(
-        PositiveIntOutputs.getSingleton(true), // weight
-        PositiveIntOutputs.getSingleton(true)  // output
+        PositiveIntOutputs.getSingleton(), // weight
+        PositiveIntOutputs.getSingleton()  // output
     );
     
     final Builder<Pair<Long,Long>> builder = new Builder<Pair<Long,Long>>(FST.INPUT_TYPE.BYTE1, outputs);
@@ -1301,7 +1300,7 @@ public class TestFSTs extends LuceneTest
     final TreeMap<String,Long> slowCompletor = new TreeMap<String,Long>();
     final TreeSet<String> allPrefixes = new TreeSet<String>();
     
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
     final IntsRef scratch = new IntsRef();
     
@@ -1416,8 +1415,8 @@ public class TestFSTs extends LuceneTest
     final TreeSet<String> allPrefixes = new TreeSet<String>();
     
     PairOutputs<Long,Long> outputs = new PairOutputs<Long,Long>(
-        PositiveIntOutputs.getSingleton(true), // weight
-        PositiveIntOutputs.getSingleton(true)  // output
+        PositiveIntOutputs.getSingleton(), // weight
+        PositiveIntOutputs.getSingleton()  // output
     );
     final Builder<Pair<Long,Long>> builder = new Builder<Pair<Long,Long>>(FST.INPUT_TYPE.BYTE1, outputs);
     final IntsRef scratch = new IntsRef();

Modified: lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestFailOnFieldCacheInsanity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestFailOnFieldCacheInsanity.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestFailOnFieldCacheInsanity.java (original)
+++ lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestFailOnFieldCacheInsanity.java Wed Jul  3 23:26:32 2013
@@ -69,6 +69,7 @@ public class TestFailOnFieldCacheInsanit
     for(Failure f : r.getFailures()) {
       if (f.getMessage().indexOf("Insane") != -1) {
         insane = true;
+        break;
       }
     }
     Assert.assertTrue(insane);

Modified: lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java (original)
+++ lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java Wed Jul  3 23:26:32 2013
@@ -650,13 +650,133 @@ public class TestPackedInts extends Luce
     wrt.set(99, (1 << 23) - 1);
     assertEquals(1 << 10, wrt.get(valueCount - 1));
     wrt.set(1, Long.MAX_VALUE);
+    wrt.set(2, -3);
+    assertEquals(64, wrt.getBitsPerValue());
     assertEquals(1 << 10, wrt.get(valueCount - 1));
     assertEquals(Long.MAX_VALUE, wrt.get(1));
+    assertEquals(-3L, wrt.get(2));
     assertEquals(2, wrt.get(4));
     assertEquals((1 << 23) - 1, wrt.get(99));
     assertEquals(10, wrt.get(7));
     assertEquals(99, wrt.get(valueCount - 10));
     assertEquals(1 << 10, wrt.get(valueCount - 1));
+    assertEquals(RamUsageEstimator.sizeOf(wrt), wrt.ramBytesUsed());
+  }
+
+  public void testPagedGrowableWriter() {
+    int pageSize = 1 << (_TestUtil.nextInt(random(), 6, 30));
+    // supports 0 values?
+    PagedGrowableWriter writer = new PagedGrowableWriter(0, pageSize, _TestUtil.nextInt(random(), 1, 64), random().nextFloat());
+    assertEquals(0, writer.size());
+
+    // compare against AppendingLongBuffer
+    AppendingLongBuffer buf = new AppendingLongBuffer();
+    int size = random().nextInt(1000000);
+    long max = 5;
+    for (int i = 0; i < size; ++i) {
+      buf.add(_TestUtil.nextLong(random(), 0, max));
+      if (rarely()) {
+        max = PackedInts.maxValue(rarely() ? _TestUtil.nextInt(random(), 0, 63) : _TestUtil.nextInt(random(), 0, 31));
+      }
+    }
+    writer = new PagedGrowableWriter(size, pageSize, _TestUtil.nextInt(random(), 1, 64), random().nextFloat());
+    assertEquals(size, writer.size());
+    for (int i = size - 1; i >= 0; --i) {
+      writer.set(i, buf.get(i));
+    }
+    for (int i = 0; i < size; ++i) {
+      assertEquals(buf.get(i), writer.get(i));
+    }
+
+    // test ramBytesUsed
+    assertEquals(RamUsageEstimator.sizeOf(writer), writer.ramBytesUsed(), 8);
+
+    // test copy
+    PagedGrowableWriter copy = writer.resize(_TestUtil.nextLong(random(), writer.size() / 2, writer.size() * 3 / 2));
+    for (long i = 0; i < copy.size(); ++i) {
+      if (i < writer.size()) {
+        assertEquals(writer.get(i), copy.get(i));
+      } else {
+        assertEquals(0, copy.get(i));
+      }
+    }
+
+    // test grow
+    PagedGrowableWriter grow = writer.grow(_TestUtil.nextLong(random(), writer.size() / 2, writer.size() * 3 / 2));
+    for (long i = 0; i < grow.size(); ++i) {
+      if (i < writer.size()) {
+        assertEquals(writer.get(i), grow.get(i));
+      } else {
+        assertEquals(0, grow.get(i));
+      }
+    }
+  }
+
+  public void testPagedMutable() {
+    final int bitsPerValue = _TestUtil.nextInt(random(), 1, 64);
+    final long max = PackedInts.maxValue(bitsPerValue);
+    int pageSize = 1 << (_TestUtil.nextInt(random(), 6, 30));
+    // supports 0 values?
+    PagedMutable writer = new PagedMutable(0, pageSize, bitsPerValue, random().nextFloat() / 2);
+    assertEquals(0, writer.size());
+
+    // compare against AppendingLongBuffer
+    AppendingLongBuffer buf = new AppendingLongBuffer();
+    int size = random().nextInt(1000000);
+    
+    for (int i = 0; i < size; ++i) {
+      buf.add(bitsPerValue == 64 ? random().nextLong() : _TestUtil.nextLong(random(), 0, max));
+    }
+    writer = new PagedMutable(size, pageSize, bitsPerValue, random().nextFloat());
+    assertEquals(size, writer.size());
+    for (int i = size - 1; i >= 0; --i) {
+      writer.set(i, buf.get(i));
+    }
+    for (int i = 0; i < size; ++i) {
+      assertEquals(buf.get(i), writer.get(i));
+    }
+
+    // test ramBytesUsed
+    assertEquals(RamUsageEstimator.sizeOf(writer) - RamUsageEstimator.sizeOf(writer.format), writer.ramBytesUsed());
+
+    // test copy
+    PagedMutable copy = writer.resize(_TestUtil.nextLong(random(), writer.size() / 2, writer.size() * 3 / 2));
+    for (long i = 0; i < copy.size(); ++i) {
+      if (i < writer.size()) {
+        assertEquals(writer.get(i), copy.get(i));
+      } else {
+        assertEquals(0, copy.get(i));
+      }
+    }
+
+    // test grow
+    PagedMutable grow = writer.grow(_TestUtil.nextLong(random(), writer.size() / 2, writer.size() * 3 / 2));
+    for (long i = 0; i < grow.size(); ++i) {
+      if (i < writer.size()) {
+        assertEquals(writer.get(i), grow.get(i));
+      } else {
+        assertEquals(0, grow.get(i));
+      }
+    }
+  }
+
+  // memory hole
+  @Ignore
+  public void testPagedGrowableWriterOverflow() {
+    final long size = _TestUtil.nextLong(random(), 2 * (long) Integer.MAX_VALUE, 3 * (long) Integer.MAX_VALUE);
+    final int pageSize = 1 << (_TestUtil.nextInt(random(), 16, 30));
+    final PagedGrowableWriter writer = new PagedGrowableWriter(size, pageSize, 1, random().nextFloat());
+    final long index = _TestUtil.nextLong(random(), (long) Integer.MAX_VALUE, size - 1);
+    writer.set(index, 2);
+    assertEquals(2, writer.get(index));
+    for (int i = 0; i < 1000000; ++i) {
+      final long idx = _TestUtil.nextLong(random(), 0, size);
+      if (idx == index) {
+        assertEquals(2, writer.get(idx));
+      } else {
+        assertEquals(0, writer.get(idx));
+      }
+    }
   }
 
   public void testSave() throws IOException {
@@ -808,13 +928,15 @@ public class TestPackedInts extends Luce
     final long[] arr = new long[RandomInts.randomIntBetween(random(), 1, 1000000)];
     for (int bpv : new int[] {0, 1, 63, 64, RandomInts.randomIntBetween(random(), 2, 62)}) {
       for (boolean monotonic : new boolean[] {true, false}) {
+        final int pageSize = 1 << _TestUtil.nextInt(random(), 6, 20);
+        final int initialPageCount = _TestUtil.nextInt(random(), 0, 16);
         AbstractAppendingLongBuffer buf;
         final int inc;
         if (monotonic) {
-          buf = new MonotonicAppendingLongBuffer();
+          buf = new MonotonicAppendingLongBuffer(initialPageCount, pageSize);
           inc = _TestUtil.nextInt(random(), -1000, 1000);
         } else {
-          buf = new AppendingLongBuffer();
+          buf = new AppendingLongBuffer(initialPageCount, pageSize);
           inc = 0;
         }
         if (bpv == 0) {

Modified: lucene/dev/branches/security/lucene/demo/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/demo/build.xml?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/demo/build.xml (original)
+++ lucene/dev/branches/security/lucene/demo/build.xml Wed Jul  3 23:26:32 2013
@@ -58,7 +58,7 @@
     <echo>Compiling XML QueryParser Demo WAR</echo>
 
     <war destfile="${build.dir}/lucene-xml-query-demo.war" webxml="src/resources/org/apache/lucene/demo/xmlparser/WEB-INF/web.xml">
-      <fileset dir="src/resources/org/apache/lucene/demo/xmlparser/"/>
+      <fileset dir="${resources.dir}/org/apache/lucene/demo/xmlparser/"/>
       <lib dir="${build.dir}">
         <include name="${final.name}.jar"/>
       </lib>

Modified: lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java (original)
+++ lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java Wed Jul  3 23:26:32 2013
@@ -95,6 +95,10 @@ public class RangeAccumulator extends Fa
         int doc = 0;
         while (doc < length && (doc = hits.bits.nextSetBit(doc)) != -1) {
           long v = ndv.get(doc);
+          // TODO: if all ranges are non-overlapping, we
+          // should instead do a bin-search up front
+          // (really, a specialized case of the interval
+          // tree)
           // TODO: use interval tree instead of linear search:
           for(int j=0;j<ranges.ranges.length;j++) {
             if (ranges.ranges[j].accept(v)) {

Modified: lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java (original)
+++ lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java Wed Jul  3 23:26:32 2013
@@ -208,9 +208,9 @@ public class DrillSideways {
           requests.add(fr);
         }
       }
-      if (requests.isEmpty()) {
-        throw new IllegalArgumentException("could not find FacetRequest for drill-sideways dimension \"" + dim + "\"");
-      }
+      // We already moved all drill-downs that didn't have a
+      // FacetRequest, in moveDrillDownOnlyClauses above:
+      assert !requests.isEmpty();
       drillSidewaysCollectors[idx++] = FacetsCollector.create(getDrillSidewaysAccumulator(dim, new FacetSearchParams(fsp.indexingParams, requests)));
     }
 
@@ -402,8 +402,13 @@ public class DrillSideways {
       query = new DrillDownQuery(filter, query);
     }
     if (sort != null) {
+      int limit = searcher.getIndexReader().maxDoc();
+      if (limit == 0) {
+        limit = 1; // the collector does not alow numHits = 0
+      }
+      topN = Math.min(topN, limit);
       final TopFieldCollector hitCollector = TopFieldCollector.create(sort,
-                                                                      Math.min(topN, searcher.getIndexReader().maxDoc()),
+                                                                      topN,
                                                                       after,
                                                                       true,
                                                                       doDocScores,
@@ -422,7 +427,12 @@ public class DrillSideways {
    */
   public DrillSidewaysResult search(ScoreDoc after,
                                     DrillDownQuery query, int topN, FacetSearchParams fsp) throws IOException {
-    TopScoreDocCollector hitCollector = TopScoreDocCollector.create(Math.min(topN, searcher.getIndexReader().maxDoc()), after, true);
+    int limit = searcher.getIndexReader().maxDoc();
+    if (limit == 0) {
+      limit = 1; // the collector does not alow numHits = 0
+    }
+    topN = Math.min(topN, limit);
+    TopScoreDocCollector hitCollector = TopScoreDocCollector.create(topN, after, true);
     DrillSidewaysResult r = search(query, hitCollector, fsp);
     return new DrillSidewaysResult(r.facetResults, hitCollector.topDocs());
   }

Modified: lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java (original)
+++ lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java Wed Jul  3 23:26:32 2013
@@ -119,17 +119,19 @@ class DrillSidewaysQuery extends Query {
             Terms terms = reader.terms(field);
             if (terms != null) {
               termsEnum = terms.iterator(null);
+            } else {
+              termsEnum = null;
             }
             lastField = field;
           }
+          dims[dim].docsEnums = new DocsEnum[drillDownTerms[dim].length];
           if (termsEnum == null) {
             nullCount++;
             continue;
           }
-          dims[dim].docsEnums = new DocsEnum[drillDownTerms[dim].length];
           for(int i=0;i<drillDownTerms[dim].length;i++) {
             if (termsEnum.seekExact(drillDownTerms[dim][i].bytes(), false)) {
-              DocsEnum docsEnum = termsEnum.docs(null, null);
+              DocsEnum docsEnum = termsEnum.docs(null, null, 0);
               if (docsEnum != null) {
                 dims[dim].docsEnums[i] = docsEnum;
                 dims[dim].maxCost = Math.max(dims[dim].maxCost, docsEnum.cost());
@@ -138,7 +140,7 @@ class DrillSidewaysQuery extends Query {
           }
         }
 
-        if (nullCount > 1) {
+        if (nullCount > 1 || (nullCount == 1 && dims.length == 1)) {
           return null;
         }
 

Modified: lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java (original)
+++ lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java Wed Jul  3 23:26:32 2013
@@ -80,7 +80,7 @@ class DrillSidewaysScorer extends Scorer
     // Position all scorers to their first matching doc:
     baseScorer.nextDoc();
     for(DocsEnumsAndFreq dim : dims) {
-      for(DocsEnum docsEnum : dim.docsEnums) {
+      for (DocsEnum docsEnum : dim.docsEnums) {
         if (docsEnum != null) {
           docsEnum.nextDoc();
         }
@@ -95,7 +95,7 @@ class DrillSidewaysScorer extends Scorer
     for(int dim=0;dim<numDims;dim++) {
       docsEnums[dim] = dims[dim].docsEnums;
       sidewaysCollectors[dim] = dims[dim].sidewaysCollector;
-      for(DocsEnum de : dims[dim].docsEnums) {
+      for (DocsEnum de : dims[dim].docsEnums) {
         if (de != null) {
           drillDownCost += de.cost();
         }

Modified: lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/CategoryPath.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/CategoryPath.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/CategoryPath.java (original)
+++ lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/CategoryPath.java Wed Jul  3 23:26:32 2013
@@ -17,6 +17,8 @@ package org.apache.lucene.facet.taxonomy
  * limitations under the License.
  */
 
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
+
 import java.util.Arrays;
 import java.util.regex.Pattern;
 
@@ -28,6 +30,18 @@ import java.util.regex.Pattern;
  */
 public class CategoryPath implements Comparable<CategoryPath> {
 
+  /*
+   * copied from DocumentWriterPerThread -- if a CategoryPath is resolved to a
+   * drill-down term which is encoded to a larger term than that length, it is
+   * silently dropped! Therefore we limit the number of characters to MAX/4 to
+   * be on the safe side.
+   */
+  /**
+   * The maximum number of characters a {@link CategoryPath} can have. That is
+   * {@link CategoryPath#toString(char)} length must not exceed that limit.
+   */
+  public final static int MAX_CATEGORY_PATH_LENGTH = (BYTE_BLOCK_SIZE - 2) / 4;
+
   /** An empty {@link CategoryPath} */
   public static final CategoryPath EMPTY = new CategoryPath();
 
@@ -63,10 +77,18 @@ public class CategoryPath implements Com
   /** Construct from the given path components. */
   public CategoryPath(final String... components) {
     assert components.length > 0 : "use CategoryPath.EMPTY to create an empty path";
+    long len = 0;
     for (String comp : components) {
       if (comp == null || comp.isEmpty()) {
         throw new IllegalArgumentException("empty or null components not allowed: " + Arrays.toString(components));
       }
+      len += comp.length();
+    }
+    len += components.length - 1; // add separators
+    if (len > MAX_CATEGORY_PATH_LENGTH) {
+      throw new IllegalArgumentException("category path exceeds maximum allowed path length: max="
+          + MAX_CATEGORY_PATH_LENGTH + " len=" + len
+          + " path=" + Arrays.toString(components).substring(0, 30) + "...");
     }
     this.components = components;
     length = components.length;
@@ -74,6 +96,12 @@ public class CategoryPath implements Com
 
   /** Construct from a given path, separating path components with {@code delimiter}. */
   public CategoryPath(final String pathString, final char delimiter) {
+    if (pathString.length() > MAX_CATEGORY_PATH_LENGTH) {
+      throw new IllegalArgumentException("category path exceeds maximum allowed path length: max="
+              + MAX_CATEGORY_PATH_LENGTH + " len=" + pathString.length()
+              + " path=" + pathString.substring(0, 30) + "...");
+    }
+
     String[] comps = pathString.split(Pattern.quote(Character.toString(delimiter)));
     if (comps.length == 1 && comps[0].isEmpty()) {
       components = null;

Modified: lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CategoryPathUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CategoryPathUtils.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CategoryPathUtils.java (original)
+++ lucene/dev/branches/security/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CategoryPathUtils.java Wed Jul  3 23:26:32 2013
@@ -39,14 +39,14 @@ class CategoryPathUtils {
    * {@link #serialize(CategoryPath, CharBlockArray)}.
    */
   public static int hashCodeOfSerialized(CharBlockArray charBlockArray, int offset) {
-    int length = (short) charBlockArray.charAt(offset++);
+    int length = charBlockArray.charAt(offset++);
     if (length == 0) {
       return 0;
     }
     
     int hash = length;
     for (int i = 0; i < length; i++) {
-      int len = (short) charBlockArray.charAt(offset++);
+      int len = charBlockArray.charAt(offset++);
       hash = hash * 31 + charBlockArray.subSequence(offset, offset + len).hashCode();
       offset += len;
     }
@@ -67,7 +67,7 @@ class CategoryPathUtils {
     }
     
     for (int i = 0; i < cp.length; i++) {
-      int len = (short) charBlockArray.charAt(offset++);
+      int len = charBlockArray.charAt(offset++);
       if (len != cp.components[i].length()) {
         return false;
       }

Modified: lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java (original)
+++ lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java Wed Jul  3 23:26:32 2013
@@ -121,8 +121,8 @@ public class TestFacetsAccumulatorWithCo
     
     // Results are ready, printing them...
     int i = 0;
-    for (FacetResult facetResult : res) {
-      if (VERBOSE) {
+    if (VERBOSE) {
+      for (FacetResult facetResult : res) {
         System.out.println("Res "+(i++)+": "+facetResult);
       }
     }

Modified: lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java (original)
+++ lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java Wed Jul  3 23:26:32 2013
@@ -28,11 +28,15 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.facet.FacetTestUtils;
+import org.apache.lucene.facet.codecs.facet42.Facet42DocValuesFormat;
 import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
@@ -58,6 +62,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField.Type;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
@@ -65,9 +70,11 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InPlaceMergeSorter;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util._TestUtil;
+import org.junit.Test;
 
 public class TestDrillSideways extends FacetTestCase {
 
@@ -426,6 +433,16 @@ public class TestDrillSideways extends F
 
     boolean canUseDV = defaultCodecSupportsSortedSet();
 
+    // TestRuleSetupAndRestoreClassEnv can sometimes
+    // randomly pick the non-general Facet42DocValuesFormat:
+    DocValuesFormat dvf = Codec.getDefault().docValuesFormat();
+    if (dvf instanceof PerFieldDocValuesFormat) {
+      dvf = ((PerFieldDocValuesFormat) dvf).getDocValuesFormatForField("$facets");
+    }
+    if (dvf instanceof Facet42DocValuesFormat) {
+      canUseDV = false;
+    }
+
     while (aChance == 0.0) {
       aChance = random().nextDouble();
     }
@@ -1144,5 +1161,33 @@ public class TestDrillSideways extends F
     }
     return b.toString();
   }
+  
+  @Test
+  public void testEmptyIndex() throws Exception {
+    // LUCENE-5045: make sure DrillSideways works with an empty index
+    Directory dir = newDirectory();
+    Directory taxoDir = newDirectory();
+    writer = new RandomIndexWriter(random(), dir);
+    taxoWriter = new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE);
+    IndexSearcher searcher = newSearcher(writer.getReader());
+    writer.close();
+    TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
+    taxoWriter.close();
+
+    // Count "Author"
+    FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(new CategoryPath("Author"), 10));
+
+    DrillSideways ds = new DrillSideways(searcher, taxoReader);
+    DrillDownQuery ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq.add(new CategoryPath("Author", "Lisa"));
+    
+    DrillSidewaysResult r = ds.search(null, ddq, 10, fsp); // this used to fail on IllegalArgEx
+    assertEquals(0, r.hits.totalHits);
+
+    r = ds.search(ddq, null, null, 10, new Sort(new SortField("foo", Type.INT)), false, false, fsp); // this used to fail on IllegalArgEx
+    assertEquals(0, r.hits.totalHits);
+    
+    IOUtils.close(searcher.getIndexReader(), taxoReader, dir, taxoDir);
+  }
 }
 

Modified: lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCategoryPath.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCategoryPath.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCategoryPath.java (original)
+++ lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCategoryPath.java Wed Jul  3 23:26:32 2013
@@ -3,6 +3,7 @@ package org.apache.lucene.facet.taxonomy
 import java.util.Arrays;
 
 import org.apache.lucene.facet.FacetTestCase;
+import org.apache.lucene.util._TestUtil;
 import org.junit.Test;
 
 /*
@@ -274,5 +275,32 @@ public class TestCategoryPath extends Fa
       // expected
     }
   }
+
+  @Test
+  public void testLongPath() throws Exception {
+    String bigComp = null;
+    while (true) {
+      int len = CategoryPath.MAX_CATEGORY_PATH_LENGTH;
+      bigComp = _TestUtil.randomSimpleString(random(), len, len);
+      if (bigComp.indexOf('\u001f') != -1) {
+        continue;
+      }
+      break;
+    }
+
+    try {
+      assertNotNull(new CategoryPath("dim", bigComp));
+      fail("long paths should not be allowed; len=" + bigComp.length());
+    } catch (IllegalArgumentException e) {
+      // expected
+    }
+
+    try {
+      assertNotNull(new CategoryPath("dim\u001f" + bigComp, '\u001f'));
+      fail("long paths should not be allowed; len=" + bigComp.length());
+    } catch (IllegalArgumentException e) {
+      // expected
+    }
+  }
   
 }

Modified: lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java (original)
+++ lucene/dev/branches/security/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java Wed Jul  3 23:26:32 2013
@@ -1,13 +1,19 @@
 package org.apache.lucene.facet.taxonomy.directory;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
 import org.apache.lucene.facet.FacetTestCase;
+import org.apache.lucene.facet.index.FacetFields;
+import org.apache.lucene.facet.params.FacetIndexingParams;
+import org.apache.lucene.facet.search.DrillDownQuery;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter.MemoryOrdinalMap;
@@ -20,8 +26,11 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util._TestUtil;
 import org.junit.Test;
 
 /*
@@ -412,5 +421,52 @@ public class TestDirectoryTaxonomyWriter
     taxoWriter.close();
     dir.close();
   }
-  
+
+  @Test
+  public void testHugeLabel() throws Exception {
+    Directory indexDir = newDirectory(), taxoDir = newDirectory();
+    IndexWriter indexWriter = new IndexWriter(indexDir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
+    DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir, OpenMode.CREATE, new Cl2oTaxonomyWriterCache(2, 1f, 1));
+    FacetFields facetFields = new FacetFields(taxoWriter);
+    
+    // Add one huge label:
+    String bigs = null;
+    int ordinal = -1;
+    CategoryPath cp = null;
+    while (true) {
+      int len = CategoryPath.MAX_CATEGORY_PATH_LENGTH - 4; // for the dimension and separator
+      bigs = _TestUtil.randomSimpleString(random(), len, len);
+      cp = new CategoryPath("dim", bigs);
+      ordinal = taxoWriter.addCategory(cp);
+      Document doc = new Document();
+      facetFields.addFields(doc, Collections.singletonList(cp));
+      indexWriter.addDocument(doc);
+      break;
+    }
+
+    // Add tiny ones to cause a re-hash
+    for (int i = 0; i < 3; i++) {
+      String s = _TestUtil.randomSimpleString(random(), 1, 10);
+      taxoWriter.addCategory(new CategoryPath("dim", s));
+      Document doc = new Document();
+      facetFields.addFields(doc, Collections.singletonList(new CategoryPath("dim", s)));
+      indexWriter.addDocument(doc);
+    }
+
+    // when too large components were allowed to be added, this resulted in a new added category
+    assertEquals(ordinal, taxoWriter.addCategory(cp));
+    
+    IOUtils.close(indexWriter, taxoWriter);
+    
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+    DrillDownQuery ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT);
+    ddq.add(cp);
+    assertEquals(1, searcher.search(ddq, 10).totalHits);
+    
+    IOUtils.close(indexReader, taxoReader);
+    
+    IOUtils.close(indexDir, taxoDir);
+  }
 }

Modified: lucene/dev/branches/security/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java (original)
+++ lucene/dev/branches/security/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java Wed Jul  3 23:26:32 2013
@@ -78,7 +78,7 @@ public class AllGroupHeadsCollectorTest 
     Document doc = new Document();
     addGroupField(doc, groupField, "author1", valueType);
     doc.add(newTextField("content", "random text", Field.Store.NO));
-    doc.add(newStringField("id_1", "1", Field.Store.NO));
+    doc.add(new IntField("id_1", 1, Field.Store.NO));
     doc.add(newStringField("id_2", "1", Field.Store.NO));
     w.addDocument(doc);
 
@@ -86,7 +86,7 @@ public class AllGroupHeadsCollectorTest 
     doc = new Document();
     addGroupField(doc, groupField, "author1", valueType);
     doc.add(newTextField("content", "some more random text blob", Field.Store.NO));
-    doc.add(newStringField("id_1", "2", Field.Store.NO));
+    doc.add(new IntField("id_1", 2, Field.Store.NO));
     doc.add(newStringField("id_2", "2", Field.Store.NO));
     w.addDocument(doc);
 
@@ -94,7 +94,7 @@ public class AllGroupHeadsCollectorTest 
     doc = new Document();
     addGroupField(doc, groupField, "author1", valueType);
     doc.add(newTextField("content", "some more random textual data", Field.Store.NO));
-    doc.add(newStringField("id_1", "3", Field.Store.NO));
+    doc.add(new IntField("id_1", 3, Field.Store.NO));
     doc.add(newStringField("id_2", "3", Field.Store.NO));
     w.addDocument(doc);
     w.commit(); // To ensure a second segment
@@ -103,7 +103,7 @@ public class AllGroupHeadsCollectorTest 
     doc = new Document();
     addGroupField(doc, groupField, "author2", valueType);
     doc.add(newTextField("content", "some random text", Field.Store.NO));
-    doc.add(newStringField("id_1", "4", Field.Store.NO));
+    doc.add(new IntField("id_1", 4, Field.Store.NO));
     doc.add(newStringField("id_2", "4", Field.Store.NO));
     w.addDocument(doc);
 
@@ -111,7 +111,7 @@ public class AllGroupHeadsCollectorTest 
     doc = new Document();
     addGroupField(doc, groupField, "author3", valueType);
     doc.add(newTextField("content", "some more random text", Field.Store.NO));
-    doc.add(newStringField("id_1", "5", Field.Store.NO));
+    doc.add(new IntField("id_1", 5, Field.Store.NO));
     doc.add(newStringField("id_2", "5", Field.Store.NO));
     w.addDocument(doc);
 
@@ -119,21 +119,21 @@ public class AllGroupHeadsCollectorTest 
     doc = new Document();
     addGroupField(doc, groupField, "author3", valueType);
     doc.add(newTextField("content", "random blob", Field.Store.NO));
-    doc.add(newStringField("id_1", "6", Field.Store.NO));
+    doc.add(new IntField("id_1", 6, Field.Store.NO));
     doc.add(newStringField("id_2", "6", Field.Store.NO));
     w.addDocument(doc);
 
     // 6 -- no author field
     doc = new Document();
     doc.add(newTextField("content", "random word stuck in alot of other text", Field.Store.NO));
-    doc.add(newStringField("id_1", "6", Field.Store.NO));
+    doc.add(new IntField("id_1", 6, Field.Store.NO));
     doc.add(newStringField("id_2", "6", Field.Store.NO));
     w.addDocument(doc);
 
     // 7 -- no author field
     doc = new Document();
     doc.add(newTextField("content", "random word stuck in alot of other text", Field.Store.NO));
-    doc.add(newStringField("id_1", "7", Field.Store.NO));
+    doc.add(new IntField("id_1", 7, Field.Store.NO));
     doc.add(newStringField("id_2", "7", Field.Store.NO));
     w.addDocument(doc);
 
@@ -406,6 +406,7 @@ public class AllGroupHeadsCollectorTest 
       for (int a : actual) {
         if (e == a) {
           found = true;
+          break;
         }
       }
 

Modified: lucene/dev/branches/security/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java (original)
+++ lucene/dev/branches/security/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java Wed Jul  3 23:26:32 2013
@@ -827,12 +827,14 @@ public class TestGrouping extends Lucene
           for(SortField sf : docSort.getSort()) {
             if (sf.getType() == SortField.Type.SCORE) {
               getScores = true;
+              break;
             }
           }
 
           for(SortField sf : groupSort.getSort()) {
             if (sf.getType() == SortField.Type.SCORE) {
               getScores = true;
+              break;
             }
           }
 

Modified: lucene/dev/branches/security/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java (original)
+++ lucene/dev/branches/security/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java Wed Jul  3 23:26:32 2013
@@ -369,7 +369,11 @@ public class PostingsHighlighter {
    *  identical to what was indexed. */
   protected String[][] loadFieldValues(IndexSearcher searcher, String[] fields, int[] docids, int maxLength) throws IOException {
     String contents[][] = new String[fields.length][docids.length];
-    LimitedStoredFieldVisitor visitor = new LimitedStoredFieldVisitor(fields, maxLength);
+    char valueSeparators[] = new char[fields.length];
+    for (int i = 0; i < fields.length; i++) {
+      valueSeparators[i] = getMultiValuedSeparator(fields[i]);
+    }
+    LimitedStoredFieldVisitor visitor = new LimitedStoredFieldVisitor(fields, valueSeparators, maxLength);
     for (int i = 0; i < docids.length; i++) {
       searcher.doc(docids[i], visitor);
       for (int j = 0; j < fields.length; j++) {
@@ -379,6 +383,16 @@ public class PostingsHighlighter {
     }
     return contents;
   }
+  
+  /** 
+   * Returns the logical separator between values for multi-valued fields.
+   * The default value is a space character, which means passages can span across values,
+   * but a subclass can override, for example with {@code U+2029 PARAGRAPH SEPARATOR (PS)}
+   * if each value holds a discrete passage for highlighting.
+   */
+  protected char getMultiValuedSeparator(String field) {
+    return ' ';
+  }
     
   private Map<Integer,String> highlightField(String field, String contents[], BreakIterator bi, BytesRef terms[], int[] docids, List<AtomicReaderContext> leaves, int maxPassages) throws IOException {  
     Map<Integer,String> highlights = new HashMap<Integer,String>();
@@ -652,12 +666,15 @@ public class PostingsHighlighter {
   
   private static class LimitedStoredFieldVisitor extends StoredFieldVisitor {
     private final String fields[];
+    private final char valueSeparators[];
     private final int maxLength;
     private final StringBuilder builders[];
     private int currentField = -1;
     
-    public LimitedStoredFieldVisitor(String fields[], int maxLength) {
+    public LimitedStoredFieldVisitor(String fields[], char valueSeparators[], int maxLength) {
+      assert fields.length == valueSeparators.length;
       this.fields = fields;
+      this.valueSeparators = valueSeparators;
       this.maxLength = maxLength;
       builders = new StringBuilder[fields.length];
       for (int i = 0; i < builders.length; i++) {
@@ -669,8 +686,8 @@ public class PostingsHighlighter {
     public void stringField(FieldInfo fieldInfo, String value) throws IOException {
       assert currentField >= 0;
       StringBuilder builder = builders[currentField];
-      if (builder.length() > 0) {
-        builder.append(' '); // for the offset gap, TODO: make this configurable
+      if (builder.length() > 0 && builder.length() < maxLength) {
+        builder.append(valueSeparators[currentField]);
       }
       if (builder.length() + value.length() > maxLength) {
         builder.append(value, 0, maxLength - builder.length());

Modified: lucene/dev/branches/security/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighter.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighter.java (original)
+++ lucene/dev/branches/security/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighter.java Wed Jul  3 23:26:32 2013
@@ -123,6 +123,43 @@ public class TestPostingsHighlighter ext
     dir.close();
   }
   
+  // simple test with multiple values that make a result longer than maxLength.
+  public void testMaxLengthWithMultivalue() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random(), MockTokenizer.SIMPLE, true));
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Document doc = new Document();
+    
+    for(int i = 0; i < 3 ; i++) {
+      Field body = new Field("body", "", offsetsType);
+      body.setStringValue("This is a multivalued field");
+      doc.add(body);
+    }
+    
+    iw.addDocument(doc);
+    
+    IndexReader ir = iw.getReader();
+    iw.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    PostingsHighlighter highlighter = new PostingsHighlighter(40);
+    Query query = new TermQuery(new Term("body", "field"));
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, topDocs);
+    assertEquals(1, snippets.length);
+    assertTrue("Snippet should have maximum 40 characters plus the pre and post tags",
+        snippets[0].length() == (40 + "<b></b>".length()));
+    
+    ir.close();
+    dir.close();
+  }
+  
   public void testMultipleFields() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random(), MockTokenizer.SIMPLE, true));
@@ -885,4 +922,48 @@ public class TestPostingsHighlighter ext
     ir.close();
     dir.close();
   }
+  
+  /** customizing the gap separator to force a sentence break */
+  public void testGapSeparator() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random(), MockTokenizer.SIMPLE, true));
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Document doc = new Document();
+    
+    Field body1 = new Field("body", "", offsetsType);
+    body1.setStringValue("This is a multivalued field");
+    doc.add(body1);
+    
+    Field body2 = new Field("body", "", offsetsType);
+    body2.setStringValue("This is something different");
+    doc.add(body2);
+    
+    iw.addDocument(doc);
+    
+    IndexReader ir = iw.getReader();
+    iw.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected char getMultiValuedSeparator(String field) {
+        assert field.equals("body");
+        return '\u2029';
+      }
+    };
+    Query query = new TermQuery(new Term("body", "field"));
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, topDocs);
+    assertEquals(1, snippets.length);
+    assertEquals("This is a multivalued <b>field</b>\u2029", snippets[0]);
+    
+    ir.close();
+    dir.close();
+  }
 }

Modified: lucene/dev/branches/security/lucene/join/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/join/build.xml?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/join/build.xml (original)
+++ lucene/dev/branches/security/lucene/join/build.xml Wed Jul  3 23:26:32 2013
@@ -1,4 +1,22 @@
 <?xml version="1.0"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one
+   or more contributor license agreements.  See the NOTICE file
+   distributed with this work for additional information
+   regarding copyright ownership.  The ASF licenses this file
+   to you under the Apache License, Version 2.0 (the
+   "License"); you may not use this file except in compliance
+   with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing,
+   software distributed under the License is distributed on an
+   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+   KIND, either express or implied.  See the License for the
+   specific language governing permissions and limitations
+   under the License.
+-->
 <project name="join" default="default">
   <description>
     Index-time and Query-time joins for normalized content

Modified: lucene/dev/branches/security/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java (original)
+++ lucene/dev/branches/security/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java Wed Jul  3 23:26:32 2013
@@ -38,6 +38,7 @@ import java.util.Comparator;
 class TermsQuery extends MultiTermQuery {
 
   private final BytesRefHash terms;
+  private final int[] ords;
   private final Query fromQuery; // Used for equals() only
 
   /**
@@ -48,6 +49,7 @@ class TermsQuery extends MultiTermQuery 
     super(field);
     this.fromQuery = fromQuery;
     this.terms = terms;
+    ords = terms.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
   }
 
   @Override
@@ -56,7 +58,7 @@ class TermsQuery extends MultiTermQuery 
       return TermsEnum.EMPTY;
     }
 
-    return new SeekingTermSetTermsEnum(terms.iterator(null), this.terms);
+    return new SeekingTermSetTermsEnum(terms.iterator(null), this.terms, ords);
   }
 
   @Override
@@ -104,12 +106,12 @@ class TermsQuery extends MultiTermQuery 
     private BytesRef seekTerm;
     private int upto = 0;
 
-    SeekingTermSetTermsEnum(TermsEnum tenum, BytesRefHash terms) {
+    SeekingTermSetTermsEnum(TermsEnum tenum, BytesRefHash terms, int[] ords) {
       super(tenum);
       this.terms = terms;
-
+      this.ords = ords;
+      comparator = BytesRef.getUTF8SortedAsUnicodeComparator();
       lastElement = terms.size() - 1;
-      ords = terms.sort(comparator = tenum.getComparator());
       lastTerm = terms.get(ords[lastElement], new BytesRef());
       seekTerm = terms.get(ords[upto], spare);
     }

Modified: lucene/dev/branches/security/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java (original)
+++ lucene/dev/branches/security/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java Wed Jul  3 23:26:32 2013
@@ -459,7 +459,7 @@ public class TestBlockJoin extends Lucen
     for(int parentDocID=0;parentDocID<numParentDocs;parentDocID++) {
       Document parentDoc = new Document();
       Document parentJoinDoc = new Document();
-      Field id = newStringField("parentID", ""+parentDocID, Field.Store.YES);
+      Field id = new IntField("parentID", parentDocID, Field.Store.YES);
       parentDoc.add(id);
       parentJoinDoc.add(id);
       parentJoinDoc.add(newStringField("isParent", "x", Field.Store.NO));
@@ -472,8 +472,8 @@ public class TestBlockJoin extends Lucen
       }
 
       if (doDeletes) {
-        parentDoc.add(newStringField("blockID", ""+parentDocID, Field.Store.NO));
-        parentJoinDoc.add(newStringField("blockID", ""+parentDocID, Field.Store.NO));
+        parentDoc.add(new IntField("blockID", parentDocID, Field.Store.NO));
+        parentJoinDoc.add(new IntField("blockID", parentDocID, Field.Store.NO));
       }
 
       final List<Document> joinDocs = new ArrayList<>();
@@ -497,7 +497,7 @@ public class TestBlockJoin extends Lucen
         Document joinChildDoc = new Document();
         joinDocs.add(joinChildDoc);
 
-        Field childID = newStringField("childID", ""+childDocID, Field.Store.YES);
+        Field childID = new IntField("childID", childDocID, Field.Store.YES);
         childDoc.add(childID);
         joinChildDoc.add(childID);
 
@@ -522,7 +522,7 @@ public class TestBlockJoin extends Lucen
         }
 
         if (doDeletes) {
-          joinChildDoc.add(newStringField("blockID", ""+parentDocID, Field.Store.NO));
+          joinChildDoc.add(new IntField("blockID", parentDocID, Field.Store.NO));
         }
 
         w.addDocument(childDoc);
@@ -541,8 +541,10 @@ public class TestBlockJoin extends Lucen
       if (VERBOSE) {
         System.out.println("DELETE parentID=" + deleteID);
       }
-      w.deleteDocuments(new Term("blockID", ""+deleteID));
-      joinW.deleteDocuments(new Term("blockID", ""+deleteID));
+      BytesRef term = new BytesRef();
+      NumericUtils.intToPrefixCodedBytes(deleteID, 0, term);
+      w.deleteDocuments(new Term("blockID", term));
+      joinW.deleteDocuments(new Term("blockID", term));
     }
 
     final IndexReader r = w.getReader();

Modified: lucene/dev/branches/security/lucene/licenses/morfologik-fsa-LICENSE-BSD.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/licenses/morfologik-fsa-LICENSE-BSD.txt?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/licenses/morfologik-fsa-LICENSE-BSD.txt (original)
+++ lucene/dev/branches/security/lucene/licenses/morfologik-fsa-LICENSE-BSD.txt Wed Jul  3 23:26:32 2013
@@ -1,6 +1,6 @@
 
 Copyright (c) 2006 Dawid Weiss
-Copyright (c) 2007-2012 Dawid Weiss, Marcin Miłkowski
+Copyright (c) 2007-2013 Dawid Weiss, Marcin Miłkowski
 All rights reserved.
 
 Redistribution and use in source and binary forms, with or without modification, 
@@ -26,4 +26,4 @@ ANY DIRECT, INDIRECT, INCIDENTAL, SPECIA
 LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON 
 ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 
 (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 
-SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file

Modified: lucene/dev/branches/security/lucene/licenses/morfologik-polish-LICENSE-BSD.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/licenses/morfologik-polish-LICENSE-BSD.txt?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/licenses/morfologik-polish-LICENSE-BSD.txt (original)
+++ lucene/dev/branches/security/lucene/licenses/morfologik-polish-LICENSE-BSD.txt Wed Jul  3 23:26:32 2013
@@ -1,62 +1,26 @@
 BSD-licensed dictionary of Polish (Morfologik)
 
-Copyright (c) 2012, Marcin Miłkowski
+Morfologik Polish dictionary.
+Version: 2.0 PoliMorf
+Copyright (c) 2013, Marcin Miłkowski
 All rights reserved.
 
-Redistribution and  use in  source and binary  forms, with  or without
-modification, are permitted provided that the following conditions are
-met:
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met: 
 
-1. Redistributions of source code must retain the above copyright
-   notice, this list of conditions and the following disclaimer.
+1. Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer. 
+2. Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution. 
 
-2. Redistributions in binary form must reproduce the above copyright
-   notice, this list of conditions and the following disclaimer in the
-   documentation and/or other materials provided with the
-   distribution.
-
-THIS SOFTWARE IS PROVIDED BY COPYRIGHT HOLDERS “AS IS” AND ANY EXPRESS
-OR  IMPLIED WARRANTIES,  INCLUDING, BUT  NOT LIMITED  TO,  THE IMPLIED
-WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
-DISCLAIMED.  IN NO EVENT  SHALL COPYRIGHT  HOLDERS OR  CONTRIBUTORS BE
-LIABLE FOR  ANY DIRECT,  INDIRECT, INCIDENTAL, SPECIAL,  EXEMPLARY, OR
-CONSEQUENTIAL DAMAGES  (INCLUDING, BUT NOT LIMITED  TO, PROCUREMENT OF
-SUBSTITUTE  GOODS OR  SERVICES;  LOSS  OF USE,  DATA,  OR PROFITS;  OR
-BUSINESS INTERRUPTION) HOWEVER CAUSED  AND ON ANY THEORY OF LIABILITY,
-WHETHER IN  CONTRACT, STRICT LIABILITY, OR  TORT (INCLUDING NEGLIGENCE
-OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN
-IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
---
-
-BSD-licensed dictionary of Polish (SGJP)
-http://sgjp.pl/morfeusz/
-
-Copyright © 2011 Zygmunt Saloni, Włodzimierz Gruszczyński, 
-	    	 Marcin Woliński, Robert Wołosz
-
-All rights reserved.
-
-Redistribution and  use in  source and binary  forms, with  or without
-modification, are permitted provided that the following conditions are
-met:
-
-1. Redistributions of source code must retain the above copyright
-   notice, this list of conditions and the following disclaimer.
-
-2. Redistributions in binary form must reproduce the above copyright
-   notice, this list of conditions and the following disclaimer in the
-   documentation and/or other materials provided with the
-   distribution.
-
-THIS SOFTWARE IS PROVIDED BY COPYRIGHT HOLDERS “AS IS” AND ANY EXPRESS
-OR  IMPLIED WARRANTIES,  INCLUDING, BUT  NOT LIMITED  TO,  THE IMPLIED
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
 WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
-DISCLAIMED.  IN NO EVENT  SHALL COPYRIGHT  HOLDERS OR  CONTRIBUTORS BE
-LIABLE FOR  ANY DIRECT,  INDIRECT, INCIDENTAL, SPECIAL,  EXEMPLARY, OR
-CONSEQUENTIAL DAMAGES  (INCLUDING, BUT NOT LIMITED  TO, PROCUREMENT OF
-SUBSTITUTE  GOODS OR  SERVICES;  LOSS  OF USE,  DATA,  OR PROFITS;  OR
-BUSINESS INTERRUPTION) HOWEVER CAUSED  AND ON ANY THEORY OF LIABILITY,
-WHETHER IN  CONTRACT, STRICT LIABILITY, OR  TORT (INCLUDING NEGLIGENCE
-OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN
-IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

Modified: lucene/dev/branches/security/lucene/licenses/morfologik-polish-NOTICE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/licenses/morfologik-polish-NOTICE.txt?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/licenses/morfologik-polish-NOTICE.txt (original)
+++ lucene/dev/branches/security/lucene/licenses/morfologik-polish-NOTICE.txt Wed Jul  3 23:26:32 2013
@@ -1,6 +1,3 @@
 
-This product includes data from BSD-licensed dictionary of Polish (Morfologik)
-(http://morfologik.blogspot.com/)
-
-This product includes data from BSD-licensed dictionary of Polish (SGJP)
-(http://sgjp.pl/morfeusz/)
+This product includes data from BSD-licensed dictionary of Polish (Morfologik, PoliMorf)
+(http://morfologik.blogspot.com/)
\ No newline at end of file

Modified: lucene/dev/branches/security/lucene/licenses/morfologik-stemming-LICENSE-BSD.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/licenses/morfologik-stemming-LICENSE-BSD.txt?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/licenses/morfologik-stemming-LICENSE-BSD.txt (original)
+++ lucene/dev/branches/security/lucene/licenses/morfologik-stemming-LICENSE-BSD.txt Wed Jul  3 23:26:32 2013
@@ -1,6 +1,6 @@
 
 Copyright (c) 2006 Dawid Weiss
-Copyright (c) 2007-2012 Dawid Weiss, Marcin Miłkowski
+Copyright (c) 2007-2013 Dawid Weiss, Marcin Miłkowski
 All rights reserved.
 
 Redistribution and use in source and binary forms, with or without modification, 
@@ -26,4 +26,4 @@ ANY DIRECT, INDIRECT, INCIDENTAL, SPECIA
 LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON 
 ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 
 (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 
-SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file

Modified: lucene/dev/branches/security/lucene/misc/src/java/org/apache/lucene/misc/SweetSpotSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/misc/src/java/org/apache/lucene/misc/SweetSpotSimilarity.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/misc/src/java/org/apache/lucene/misc/SweetSpotSimilarity.java (original)
+++ lucene/dev/branches/security/lucene/misc/src/java/org/apache/lucene/misc/SweetSpotSimilarity.java Wed Jul  3 23:26:32 2013
@@ -158,7 +158,7 @@ public class SweetSpotSimilarity extends
    * @see #baselineTf
    */
   @Override
-  public float tf(int freq) {
+  public float tf(float freq) {
     return baselineTf(freq);
   }
   

Modified: lucene/dev/branches/security/lucene/misc/src/java/org/apache/lucene/util/fst/ListOfOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/misc/src/java/org/apache/lucene/util/fst/ListOfOutputs.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/misc/src/java/org/apache/lucene/util/fst/ListOfOutputs.java (original)
+++ lucene/dev/branches/security/lucene/misc/src/java/org/apache/lucene/util/fst/ListOfOutputs.java Wed Jul  3 23:26:32 2013
@@ -38,6 +38,15 @@ import org.apache.lucene.util.IntsRef; /
  * more than one output, as this requires pushing all
  * multi-output values to a final state.
  *
+ * <p>NOTE: the only way to create multiple outputs is to
+ * add the same input to the FST multiple times in a row.  This is
+ * how the FST maps a single input to multiple outputs (e.g. you
+ * cannot pass a List&lt;Object&gt; to {@link Builder#add}).  If
+ * your outputs are longs, and you need at most 2, then use
+ * {@link UpToTwoPositiveIntOutputs} instead since it stores
+ * the outputs more compactly (by stealing a bit from each
+ * long value).
+ *
  * <p>NOTE: this cannot wrap itself (ie you cannot make an
  * FST with List&lt;List&lt;Object&gt;&gt; outputs using this).
  *

Modified: lucene/dev/branches/security/lucene/misc/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/misc/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/misc/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java (original)
+++ lucene/dev/branches/security/lucene/misc/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java Wed Jul  3 23:26:32 2013
@@ -17,21 +17,6 @@ package org.apache.lucene.util.fst;
  * limitations under the License.
  */
 
-/**
- * An FST {@link Outputs} implementation where each output
- * is one or two non-negative long values.  If it's a
- * single output, Long is returned; else, TwoLongs.  Order
- * is preserved in the TwoLongs case, ie .first is the first
- * input/output added to Builder, and .second is the
- * second.  You cannot store 0 output with this (that's
- * reserved to mean "no output")!
- *
- * NOTE: the resulting FST is not guaranteed to be minimal!
- * See {@link Builder}.
- *
- * @lucene.experimental
- */
-
 import java.io.IOException;
 
 import org.apache.lucene.store.DataInput;
@@ -46,11 +31,21 @@ import org.apache.lucene.store.DataOutpu
  * second.  You cannot store 0 output with this (that's
  * reserved to mean "no output")!
  *
- * NOTE: the resulting FST is not guaranteed to be minimal!
+ * <p>NOTE: the only way to create a TwoLongs output is to
+ * add the same input to the FST twice in a row.  This is
+ * how the FST maps a single input to two outputs (e.g. you
+ * cannot pass a TwoLongs to {@link Builder#add}.  If you
+ * need more than two then use {@link ListOfOutputs}, but if
+ * you only have at most 2 then this implementation will
+ * require fewer bytes as it steals one bit from each long
+ * value.
+ *
+ * <p>NOTE: the resulting FST is not guaranteed to be minimal!
  * See {@link Builder}.
  *
  * @lucene.experimental
  */
+
 public final class UpToTwoPositiveIntOutputs extends Outputs<Object> {
 
   /** Holds two long outputs. */

Modified: lucene/dev/branches/security/lucene/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java (original)
+++ lucene/dev/branches/security/lucene/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java Wed Jul  3 23:26:32 2013
@@ -41,7 +41,7 @@ public class TestIndexSplitter extends L
       ((MockDirectoryWrapper)fsDir).setAssertNoUnrefencedFilesOnClose(false);
     }
 
-    LogMergePolicy mergePolicy = new LogByteSizeMergePolicy();
+    MergePolicy mergePolicy = new LogByteSizeMergePolicy();
     mergePolicy.setNoCFSRatio(1.0);
     mergePolicy.setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
     IndexWriter iw = new IndexWriter(

Modified: lucene/dev/branches/security/lucene/misc/src/test/org/apache/lucene/index/sorter/SorterTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/misc/src/test/org/apache/lucene/index/sorter/SorterTestBase.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/misc/src/test/org/apache/lucene/index/sorter/SorterTestBase.java (original)
+++ lucene/dev/branches/security/lucene/misc/src/test/org/apache/lucene/index/sorter/SorterTestBase.java Wed Jul  3 23:26:32 2013
@@ -98,13 +98,8 @@ public abstract class SorterTestBase ext
     }
     
     @Override
-    public ExactSimScorer exactSimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
-      return in.exactSimScorer(weight, context);
-    }
-    
-    @Override
-    public SloppySimScorer sloppySimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
-      return in.sloppySimScorer(weight, context);
+    public SimScorer simScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
+      return in.simScorer(weight, context);
     }
     
   }

Modified: lucene/dev/branches/security/lucene/misc/src/test/org/apache/lucene/misc/SweetSpotSimilarityTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/misc/src/test/org/apache/lucene/misc/SweetSpotSimilarityTest.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/misc/src/test/org/apache/lucene/misc/SweetSpotSimilarityTest.java (original)
+++ lucene/dev/branches/security/lucene/misc/src/test/org/apache/lucene/misc/SweetSpotSimilarityTest.java Wed Jul  3 23:26:32 2013
@@ -246,7 +246,7 @@ public class SweetSpotSimilarityTest ext
   
     SweetSpotSimilarity ss = new SweetSpotSimilarity() {
         @Override
-        public float tf(int freq) {
+        public float tf(float freq) {
           return hyperbolicTf(freq);
         }
       };

Modified: lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/MaxFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/MaxFloatFunction.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/MaxFloatFunction.java (original)
+++ lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/MaxFloatFunction.java Wed Jul  3 23:26:32 2013
@@ -27,23 +27,18 @@ public class MaxFloatFunction extends Mu
   public MaxFloatFunction(ValueSource[] sources) {
     super(sources);
   }
-
-  @Override  
+  
+  @Override
   protected String name() {
     return "max";
   }
 
   @Override
   protected float func(int doc, FunctionValues[] valsArr) {
-    boolean first = true;
-    float val = 0.0f;
+    if (valsArr.length == 0) return 0.0f;
+    float val = Float.NEGATIVE_INFINITY;
     for (FunctionValues vals : valsArr) {
-      if (first) {
-        first = false;
-        val = vals.floatVal(doc);
-      } else {
-        val = Math.max(vals.floatVal(doc),val);
-      }
+      val = Math.max(vals.floatVal(doc), val);
     }
     return val;
   }

Modified: lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/MinFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/MinFloatFunction.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/MinFloatFunction.java (original)
+++ lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/MinFloatFunction.java Wed Jul  3 23:26:32 2013
@@ -35,15 +35,10 @@ public class MinFloatFunction extends Mu
 
   @Override
   protected float func(int doc, FunctionValues[] valsArr) {
-    boolean first = true;
-    float val = 0.0f;
+    if (valsArr.length == 0) return 0.0f;
+    float val = Float.POSITIVE_INFINITY;
     for (FunctionValues vals : valsArr) {
-      if (first) {
-        first = false;
-        val = vals.floatVal(doc);
-      } else {
-        val = Math.min(vals.floatVal(doc),val);
-      }
+      val = Math.min(vals.floatVal(doc), val);
     }
     return val;
   }

Modified: lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java (original)
+++ lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java Wed Jul  3 23:26:32 2013
@@ -29,7 +29,7 @@ import java.io.IOException;
 import java.util.Map;
 
 /** 
- * Function that returns {@link TFIDFSimilarity#decodeNormValue(byte)}
+ * Function that returns {@link TFIDFSimilarity#decodeNormValue(long)}
  * for every document.
  * <p>
  * Note that the configured Similarity for the field must be

Modified: lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java (original)
+++ lucene/dev/branches/security/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java Wed Jul  3 23:26:32 2013
@@ -29,7 +29,7 @@ import java.io.IOException;
 import java.util.Map;
 
 /** 
- * Function that returns {@link TFIDFSimilarity#tf(int)}
+ * Function that returns {@link TFIDFSimilarity#tf(float)}
  * for every document.
  * <p>
  * Note that the configured Similarity for the field must be

Modified: lucene/dev/branches/security/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java (original)
+++ lucene/dev/branches/security/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java Wed Jul  3 23:26:32 2013
@@ -53,26 +53,6 @@ public class TestCustomScoreQuery extend
   }
 
   /**
-   * Test that CustomScoreQuery of Type.BYTE returns the expected scores.
-   */
-  @Test
-  public void testCustomScoreByte() throws Exception {
-    // INT field values are small enough to be parsed as byte
-    doTestCustomScore(BYTE_VALUESOURCE, 1.0);
-    doTestCustomScore(BYTE_VALUESOURCE, 2.0);
-  }
-
-  /**
-   * Test that CustomScoreQuery of Type.SHORT returns the expected scores.
-   */
-  @Test
-  public void testCustomScoreShort() throws Exception {
-    // INT field values are small enough to be parsed as short
-    doTestCustomScore(SHORT_VALUESOURCE, 1.0);
-    doTestCustomScore(SHORT_VALUESOURCE, 3.0);
-  }
-
-  /**
    * Test that CustomScoreQuery of Type.INT returns the expected scores.
    */
   @Test