You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2014/12/14 23:50:23 UTC

svn commit: r1645537 [2/7] - in /lucene/dev/branches/lucene2878: ./ lucene/ lucene/benchmark/ lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/ lucene/core/ lucene/core/src...

Modified: lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java Sun Dec 14 22:50:20 2014
@@ -435,16 +435,16 @@ public class TestIndexWriterReader exten
     void close(boolean doWait) throws Throwable {
       didClose = true;
       if (doWait) {
-        mainWriter.waitForMerges();
+        mainWriter.close();
       } else {
-        mainWriter.abortMerges();
+        mainWriter.rollback();
       }
-      mainWriter.close();
     }
 
     void closeDir() throws Throwable {
-      for (int i = 0; i < numDirs; i++)
+      for (int i = 0; i < numDirs; i++) {
         readers[i].close();
+      }
       addDir.close();
     }
     

Modified: lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java Sun Dec 14 22:50:20 2014
@@ -18,9 +18,13 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
+import java.lang.reflect.Field;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -28,11 +32,15 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SerialMergeScheduler;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.RamUsageTester;
 
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 
@@ -50,6 +58,15 @@ public class TestLRUFilterCache extends
 
   };
 
+  public void testFilterRamBytesUsed() {
+    final Filter simpleFilter = new QueryWrapperFilter(new TermQuery(new Term("some_field", "some_term")));
+    final long actualRamBytesUsed = RamUsageTester.sizeOf(simpleFilter);
+    final long ramBytesUsed = LRUFilterCache.FILTER_DEFAULT_RAM_BYTES_USED;
+    // we cannot assert exactly that the constant is correct since actual
+    // memory usage depends on JVM implementations and settings (eg. UseCompressedOops)
+    assertEquals(actualRamBytesUsed, ramBytesUsed, actualRamBytesUsed / 2);
+  }
+
   public void testConcurrency() throws Throwable {
     final LRUFilterCache filterCache = new LRUFilterCache(1 + random().nextInt(20), 1 + random().nextInt(10000));
     Directory dir = newDirectory();
@@ -151,32 +168,233 @@ public class TestLRUFilterCache extends
     final Filter red = new QueryWrapperFilter(new TermQuery(new Term("color", "red")));
     final Filter green = new QueryWrapperFilter(new TermQuery(new Term("color", "green")));
 
-    assertEquals(Collections.emptySet(), filterCache.cachedFilters());
+    assertEquals(Collections.emptyList(), filterCache.cachedFilters());
 
     // the filter is not cached on any segment: no changes
     searcher.search(new ConstantScoreQuery(filterCache.doCache(green, NEVER_CACHE)), 1);
-    assertEquals(Collections.emptySet(), filterCache.cachedFilters());
+    assertEquals(Collections.emptyList(), filterCache.cachedFilters());
 
     searcher.search(new ConstantScoreQuery(filterCache.doCache(red, FilterCachingPolicy.ALWAYS_CACHE)), 1);
-    assertEquals(Collections.singleton(red), filterCache.cachedFilters());
+    assertEquals(Collections.singletonList(red), filterCache.cachedFilters());
 
     searcher.search(new ConstantScoreQuery(filterCache.doCache(green, FilterCachingPolicy.ALWAYS_CACHE)), 1);
-    assertEquals(new HashSet<>(Arrays.asList(red, green)), filterCache.cachedFilters());
+    assertEquals(Arrays.asList(red, green), filterCache.cachedFilters());
 
     searcher.search(new ConstantScoreQuery(filterCache.doCache(red, FilterCachingPolicy.ALWAYS_CACHE)), 1);
-    assertEquals(new HashSet<>(Arrays.asList(red, green)), filterCache.cachedFilters());
+    assertEquals(Arrays.asList(green, red), filterCache.cachedFilters());
 
     searcher.search(new ConstantScoreQuery(filterCache.doCache(blue, FilterCachingPolicy.ALWAYS_CACHE)), 1);
-    assertEquals(new HashSet<>(Arrays.asList(red, blue)), filterCache.cachedFilters());
+    assertEquals(Arrays.asList(red, blue), filterCache.cachedFilters());
 
     searcher.search(new ConstantScoreQuery(filterCache.doCache(blue, FilterCachingPolicy.ALWAYS_CACHE)), 1);
-    assertEquals(new HashSet<>(Arrays.asList(red, blue)), filterCache.cachedFilters());
+    assertEquals(Arrays.asList(red, blue), filterCache.cachedFilters());
 
     searcher.search(new ConstantScoreQuery(filterCache.doCache(green, FilterCachingPolicy.ALWAYS_CACHE)), 1);
-    assertEquals(new HashSet<>(Arrays.asList(green, blue)), filterCache.cachedFilters());
+    assertEquals(Arrays.asList(blue, green), filterCache.cachedFilters());
 
     searcher.search(new ConstantScoreQuery(filterCache.doCache(red, NEVER_CACHE)), 1);
-    assertEquals(new HashSet<>(Arrays.asList(green, blue)), filterCache.cachedFilters());
+    assertEquals(Arrays.asList(blue, green), filterCache.cachedFilters());
+
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testCache() throws IOException {
+    Directory dir = newDirectory();
+    final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+    Document doc = new Document();
+    StringField f = new StringField("color", "", Store.NO);
+    doc.add(f);
+    final int numDocs = atLeast(10);
+    for (int i = 0; i < numDocs; ++i) {
+      f.setStringValue(RandomPicks.randomFrom(random(), Arrays.asList("blue", "red", "green")));
+      w.addDocument(doc);
+    }
+    final DirectoryReader reader = w.getReader();
+    final LeafReaderContext leaf1 = reader.leaves().get(0);
+
+    Filter filter1 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
+    // different instance yet equal
+    Filter filter2 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
+
+    final LRUFilterCache filterCache = new LRUFilterCache(Integer.MAX_VALUE, Long.MAX_VALUE);
+    final Filter cachedFilter1 = filterCache.doCache(filter1, FilterCachingPolicy.ALWAYS_CACHE);
+    DocIdSet cached1 = cachedFilter1.getDocIdSet(leaf1, null);
+
+    final Filter cachedFilter2 = filterCache.doCache(filter2, NEVER_CACHE);
+    DocIdSet cached2 = cachedFilter2.getDocIdSet(leaf1, null);
+    assertSame(cached1, cached2);
+
+    filterCache.assertConsistent();
+
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testClearFilter() throws IOException {
+    Directory dir = newDirectory();
+    final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+    Document doc = new Document();
+    StringField f = new StringField("color", "", Store.NO);
+    doc.add(f);
+    final int numDocs = atLeast(10);
+    for (int i = 0; i < numDocs; ++i) {
+      f.setStringValue(random().nextBoolean() ? "red" : "blue");
+      w.addDocument(doc);
+    }
+    final DirectoryReader reader = w.getReader();
+    final LeafReaderContext leaf1 = reader.leaves().get(0);
+
+    final Filter filter1 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
+    // different instance yet equal
+    final Filter filter2 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
+
+    final LRUFilterCache filterCache = new LRUFilterCache(Integer.MAX_VALUE, Long.MAX_VALUE);
+
+    final Filter cachedFilter1 = filterCache.doCache(filter1, FilterCachingPolicy.ALWAYS_CACHE);
+    cachedFilter1.getDocIdSet(leaf1, null);
+
+    filterCache.clearFilter(filter2);
+
+    assertTrue(filterCache.cachedFilters().isEmpty());
+    filterCache.assertConsistent();
+
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  // This test makes sure that by making the same assumptions as LRUFilterCache, RAMUsageTester
+  // computes the same memory usage.
+  public void testRamBytesUsedAgreesWithRamUsageTester() throws IOException {
+    final LRUFilterCache filterCache = new LRUFilterCache(1 + random().nextInt(5), 1 + random().nextInt(10000));
+    // an accumulator that only sums up memory usage of referenced filters and doc id sets
+    final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {
+      @Override
+      public long accumulateObject(Object o, long shallowSize, Map<Field,Object> fieldValues, Collection<Object> queue) {
+        if (o instanceof DocIdSet) {
+          return ((DocIdSet) o).ramBytesUsed();
+        }
+        if (o instanceof Filter) {
+          return filterCache.ramBytesUsed((Filter) o);
+        }
+        if (o.getClass().getSimpleName().equals("SegmentCoreReaders")) {
+          // do not take core cache keys into account
+          return 0;
+        }
+        if (o instanceof Map) {
+          Map<?,?> map = (Map<?,?>) o;
+          queue.addAll(map.keySet());
+          queue.addAll(map.values());
+          final long sizePerEntry = o instanceof LinkedHashMap
+              ? LRUFilterCache.LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY
+              : LRUFilterCache.HASHTABLE_RAM_BYTES_PER_ENTRY;
+          return sizePerEntry * map.size();
+        }
+        // follow links to other objects, but ignore their memory usage
+        super.accumulateObject(o, shallowSize, fieldValues, queue);
+        return  0;
+      }
+      @Override
+      public long accumulateArray(Object array, long shallowSize, List<Object> values, Collection<Object> queue) {
+        // follow links to other objects, but ignore their memory usage
+        super.accumulateArray(array, shallowSize, values, queue);
+        return 0;
+      }
+    };
+
+    Directory dir = newDirectory();
+    // serial merges so that segments do not get closed while we are measuring ram usage
+    // with RamUsageTester
+    IndexWriterConfig iwc = newIndexWriterConfig().setMergeScheduler(new SerialMergeScheduler());
+    final RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+
+    final List<String> colors = Arrays.asList("blue", "red", "green", "yellow");
+
+    Document doc = new Document();
+    StringField f = new StringField("color", "", Store.NO);
+    doc.add(f);
+    final int iters = atLeast(5);
+    for (int iter = 0; iter < iters; ++iter) {
+      final int numDocs = atLeast(10);
+      for (int i = 0; i < numDocs; ++i) {
+        f.setStringValue(RandomPicks.randomFrom(random(), colors));
+        w.addDocument(doc);
+      }
+      try (final DirectoryReader reader = w.getReader()) {
+        final IndexSearcher searcher = new IndexSearcher(reader);
+        for (int i = 0; i < 3; ++i) {
+          final Filter filter = new QueryWrapperFilter(new TermQuery(new Term("color", RandomPicks.randomFrom(random(), colors))));
+          searcher.search(new ConstantScoreQuery(filterCache.doCache(filter, MAYBE_CACHE_POLICY)), 1);
+        }
+      }
+      filterCache.assertConsistent();
+      assertEquals(RamUsageTester.sizeOf(filterCache, acc), filterCache.ramBytesUsed());
+    }
+
+    w.close();
+    dir.close();
+  }
+
+  /** A filter that produces empty sets. */
+  private static class DummyFilter extends Filter {
+
+    @Override
+    public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
+      return null;
+    }
+
+  }
+
+  // Test what happens when the cache contains only filters and doc id sets
+  // that require very little memory. In that case most of the memory is taken
+  // by the cache itself, not cache entries, and we want to make sure that
+  // memory usage is not grossly underestimated.
+  public void testRamBytesUsedConstantEntryOverhead() throws IOException {
+    final LRUFilterCache filterCache = new LRUFilterCache(1000000, 10000000);
+
+    final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {
+      @Override
+      public long accumulateObject(Object o, long shallowSize, Map<Field,Object> fieldValues, Collection<Object> queue) {
+        if (o instanceof DocIdSet) {
+          return ((DocIdSet) o).ramBytesUsed();
+        }
+        if (o instanceof Filter) {
+          return filterCache.ramBytesUsed((Filter) o);
+        }
+        if (o.getClass().getSimpleName().equals("SegmentCoreReaders")) {
+          // do not follow references to core cache keys
+          return 0;
+        }
+        return super.accumulateObject(o, shallowSize, fieldValues, queue);
+      }
+    };
+
+    Directory dir = newDirectory();
+    final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      w.addDocument(doc);
+    }
+    final DirectoryReader reader = w.getReader();
+    final IndexSearcher searcher = new IndexSearcher(reader);
+
+    final int numFilters = atLeast(1000);
+    for (int i = 0; i < numFilters; ++i) {
+      final Filter filter = new DummyFilter();
+      final Filter cached = filterCache.doCache(filter, FilterCachingPolicy.ALWAYS_CACHE);
+      searcher.search(new ConstantScoreQuery(cached), 1);
+    }
+
+    final long actualRamBytesUsed = RamUsageTester.sizeOf(filterCache, acc);
+    final long expectedRamBytesUsed = filterCache.ramBytesUsed();
+    // error < 30%
+    assertEquals(actualRamBytesUsed, expectedRamBytesUsed, 30 * actualRamBytesUsed / 100);
 
     reader.close();
     w.close();

Modified: lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java Sun Dec 14 22:50:20 2014
@@ -462,7 +462,9 @@ public class TestSearcherManager extends
             new FilterDirectoryReader.SubReaderWrapper() {
               @Override
               public LeafReader wrap(LeafReader reader) {
-                return new MyFilterLeafReader(reader);
+                FilterLeafReader wrapped = new MyFilterLeafReader(reader);
+                assertEquals(reader, wrapped.getDelegate());
+                return wrapped;
               }
             });
     }
@@ -477,7 +479,12 @@ public class TestSearcherManager extends
   public void testCustomDirectoryReader() throws Exception {
     Directory dir = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random(), dir);
-    DirectoryReader reader = new MyFilterDirectoryReader(w.getReader());
+    DirectoryReader nrtReader = w.getReader();
+
+    FilterDirectoryReader reader = new MyFilterDirectoryReader(nrtReader);
+    assertEquals(nrtReader, reader.getDelegate());
+    assertEquals(nrtReader, FilterDirectoryReader.unwrap(reader));
+
     SearcherManager mgr = new SearcherManager(reader, null);
     for(int i=0;i<10;i++) {
       w.addDocument(new Document());

Modified: lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/store/TestFilterDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/store/TestFilterDirectory.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/store/TestFilterDirectory.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/store/TestFilterDirectory.java Sun Dec 14 22:50:20 2014
@@ -17,6 +17,7 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.HashSet;
 import java.util.Set;
@@ -39,5 +40,12 @@ public class TestFilterDirectory extends
       }
     }
   }
-  
+
+  public void testUnwrap() throws IOException {
+    Directory dir = FSDirectory.open(createTempDir());
+    FilterDirectory dir2 = new FilterDirectory(dir);
+    assertEquals(dir, dir2.getDelegate());
+    assertEquals(dir, FilterDirectory.unwrap(dir2));
+    dir2.close();
+  }
 }

Modified: lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Sun Dec 14 22:50:20 2014
@@ -1593,4 +1593,51 @@ public class TestFSTs extends LuceneTest
     }
   }
 
+  public void testIllegallyModifyRootArc() throws Exception {
+    assumeTrue("test relies on assertions", assertsAreEnabled);
+
+    Set<BytesRef> terms = new HashSet<>();
+    for(int i=0;i<100;i++) {
+      String prefix = Character.toString((char) ('a' + i));
+      terms.add(new BytesRef(prefix));
+      if (prefix.equals("m") == false) {
+        for(int j=0;j<20;j++) {
+          // Make a big enough FST that the root cache will be created:
+          String suffix = TestUtil.randomRealisticUnicodeString(random(), 10, 20);
+          terms.add(new BytesRef(prefix + suffix));
+        }
+      }
+    }
+
+    List<BytesRef> termsList = new ArrayList<>(terms);
+    Collections.sort(termsList);
+
+    ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
+    Builder<BytesRef> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
+
+    IntsRefBuilder input = new IntsRefBuilder();
+    for(BytesRef term : termsList) {
+      Util.toIntsRef(term, input);
+      builder.add(input.get(), term);
+    }
+
+    FST<BytesRef> fst = builder.finish();
+    
+    Arc<BytesRef> arc = new FST.Arc<>();
+    fst.getFirstArc(arc);
+    FST.BytesReader reader = fst.getBytesReader();
+    arc = fst.findTargetArc((int) 'm', arc, arc, reader);
+    assertNotNull(arc);
+    assertEquals(new BytesRef("m"), arc.output);
+
+    // NOTE: illegal:
+    arc.output.length = 0;
+
+    fst.getFirstArc(arc);
+    try {
+      arc = fst.findTargetArc((int) 'm', arc, arc, reader);
+    } catch (AssertionError ae) {
+      // expected
+    }
+  }
 }

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/NumberRangePrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/NumberRangePrefixTreeStrategy.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/NumberRangePrefixTreeStrategy.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/NumberRangePrefixTreeStrategy.java Sun Dec 14 22:50:20 2014
@@ -25,10 +25,11 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
 import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree;
 
-import java.text.ParseException;
-
 /** A PrefixTree based on Number/Date ranges. This isn't very "spatial" on the surface (to the user) but
- * it's implemented using spatial so that's why it's here extending a SpatialStrategy.
+ * it's implemented using spatial so that's why it's here extending a SpatialStrategy. When using this class, you will
+ * use various utility methods on the prefix tree implementation to convert objects/strings to/from shapes.
+ *
+ * To use with dates, pass in {@link org.apache.lucene.spatial.prefix.tree.DateRangePrefixTree}.
  *
  * @lucene.experimental
  */
@@ -55,22 +56,6 @@ public class NumberRangePrefixTreeStrate
     return new Field[]{field};
   }
 
-  /** For a Date based tree, pass in a Calendar, with unspecified fields marked as cleared.
-   * See {@link NumberRangePrefixTree#toShape(Object)}. */
-  public Shape toShape(Object value) {
-    return getGrid().toShape(value);
-  }
-
-  /** See {@link NumberRangePrefixTree#toRangeShape(Shape, Shape)}. */
-  public Shape toRangeShape(Shape min, Shape max) {
-    return getGrid().toRangeShape(min, max);
-  }
-
-  /** See {@link NumberRangePrefixTree#parseShape(String)}. */
-  public Shape parseShape(String str) throws ParseException {
-    return getGrid().parseShape(str);
-  }
-
   /** Unsupported. */
   @Override
   public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java Sun Dec 14 22:50:20 2014
@@ -17,8 +17,6 @@ package org.apache.lucene.spatial.prefix
  * limitations under the License.
  */
 
-import com.spatial4j.core.shape.Shape;
-
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.Calendar;
@@ -27,11 +25,13 @@ import java.util.GregorianCalendar;
 import java.util.Locale;
 import java.util.TimeZone;
 
+import com.spatial4j.core.shape.Shape;
+
 /**
  * A PrefixTree for date ranges in which the levels of the tree occur at natural periods of time (e.g. years,
  * months, ...). You pass in {@link Calendar} objects with the desired fields set and the unspecified
- * fields unset, which conveys the precision.  The implementation tries to be generic to the Calendar
- * abstraction, making some optimizations when a Gregorian is used, but no others have been tested.
+ * fields unset, which conveys the precision.  The implementation makes some optimization assumptions about a
+ * {@link java.util.GregorianCalendar}; others could probably be supported easily.
  * <p/>
  * Warning: If you construct a Calendar and then get something from the object like a field (e.g. year) or
  * milliseconds, then every field is fully set by side-effect. So after setting the fields, pass it to this
@@ -42,7 +42,7 @@ public class DateRangePrefixTree extends
 
   /*
     WARNING  java.util.Calendar is tricky to work with:
-    * If you "get" any field value, every fields because "set". This can introduce a Heisenbug effect,
+    * If you "get" any field value, every field becomes "set". This can introduce a Heisenbug effect,
         when in a debugger in some cases. Fortunately, Calendar.toString() doesn't apply.
     * Beware Calendar underflow of the underlying long.  If you create a Calendar from LONG.MIN_VALUE, and clear
      a field, it will underflow and appear close to LONG.MAX_VALUE (BC to AD).
@@ -94,10 +94,10 @@ public class DateRangePrefixTree extends
 
   public static final DateRangePrefixTree INSTANCE = new DateRangePrefixTree();
 
-  private final LevelledValue minLV, maxLV;
-  private final LevelledValue gregorianChangeDateLV;
+  private final UnitNRShape minLV, maxLV;
+  private final UnitNRShape gregorianChangeDateLV;
 
-  private DateRangePrefixTree() {
+  protected DateRangePrefixTree() {
     super(new int[]{//sublevels by level
         NUM_MYEARS,
         1000,//1 thousand thousand-years in a million years
@@ -109,27 +109,27 @@ public class DateRangePrefixTree extends
         calFieldLen(Calendar.SECOND),
         calFieldLen(Calendar.MILLISECOND),
     });
-    maxLV = (LevelledValue) toShape((Calendar)MAXCAL.clone());
-    minLV = (LevelledValue) toShape((Calendar)MINCAL.clone());
+    maxLV = toShape((Calendar)MAXCAL.clone());
+    minLV = toShape((Calendar)MINCAL.clone());
     if (MAXCAL instanceof GregorianCalendar) {
-      //TODO this should be a configurable param by passing a Calendar surving as a template.
+      //TODO this should be a configurable param by passing a Calendar serving as a template.
       GregorianCalendar gCal = (GregorianCalendar)MAXCAL;
-      gregorianChangeDateLV = (LevelledValue) toShape(gCal.getGregorianChange());
+      gregorianChangeDateLV = toUnitShape(gCal.getGregorianChange());
     } else {
       gregorianChangeDateLV = null;
     }
   }
 
   @Override
-  protected int getNumSubCells(LevelledValue lv) {
-    int cmp = comparePrefixLV(lv, maxLV);
+  public int getNumSubCells(UnitNRShape lv) {
+    int cmp = comparePrefix(lv, maxLV);
     assert cmp <= 0;
     if (cmp == 0)//edge case (literally!)
       return maxLV.getValAtLevel(lv.getLevel()+1);
 
     // if using GregorianCalendar and we're after the "Gregorian change date" then we'll compute
     //  the sub-cells ourselves more efficiently without the need to construct a Calendar.
-    cmp = gregorianChangeDateLV != null ? comparePrefixLV(lv, gregorianChangeDateLV) : -1;
+    cmp = gregorianChangeDateLV != null ? comparePrefix(lv, gregorianChangeDateLV) : -1;
     //TODO consider also doing fast-path if field is <= hours even if before greg change date
     if (cmp >= 0) {
       int result = fastSubCells(lv);
@@ -140,7 +140,7 @@ public class DateRangePrefixTree extends
     }
   }
 
-  private int fastSubCells(LevelledValue lv) {
+  private int fastSubCells(UnitNRShape lv) {
     if (lv.getLevel() == yearLevel+1) {//month
       switch (lv.getValAtLevel(lv.getLevel())) {
         case Calendar.SEPTEMBER:
@@ -166,12 +166,12 @@ public class DateRangePrefixTree extends
     }
   }
 
-  private int slowSubCells(LevelledValue lv) {
+  private int slowSubCells(UnitNRShape lv) {
     int field = FIELD_BY_LEVEL[lv.getLevel()+1];
     //short-circuit optimization (GregorianCalendar assumptions)
     if (field == -1 || field == Calendar.YEAR || field >= Calendar.HOUR_OF_DAY)//TODO make configurable
       return super.getNumSubCells(lv);
-    Calendar cal = toCalendarLV(lv);//somewhat heavyweight op; ideally should be stored on LevelledValue somehow
+    Calendar cal = toCalendar(lv);//somewhat heavyweight op; ideally should be stored on UnitNRShape somehow
     return cal.getActualMaximum(field) - cal.getActualMinimum(field) + 1;
   }
 
@@ -182,6 +182,21 @@ public class DateRangePrefixTree extends
   }
 
   /** Calendar utility method:
+   * Returns the spatial prefix tree level for the corresponding {@link java.util.Calendar} field, such as
+   * {@link java.util.Calendar#YEAR}.  If there's no match, the next greatest level is returned as a negative value.
+   */
+  public int getTreeLevelForCalendarField(int calField) {
+    for (int i = yearLevel; i < FIELD_BY_LEVEL.length; i++) {
+      if (FIELD_BY_LEVEL[i] == calField) {
+        return i;
+      } else if (FIELD_BY_LEVEL[i] > calField) {
+        return -1 * i;
+      }
+    }
+    throw new IllegalArgumentException("Bad calendar field?: " + calField);
+  }
+
+  /** Calendar utility method:
    * Gets the Calendar field code of the last field that is set prior to an unset field. It only
    * examines fields relevant to the prefix tree. If no fields are set, it returns -1. */
   public int getCalPrecisionField(Calendar cal) {
@@ -214,7 +229,7 @@ public class DateRangePrefixTree extends
    * result in a {@link java.lang.IllegalArgumentException}.
    */
   @Override
-  public Shape toShape(Object value) {
+  public UnitNRShape toUnitShape(Object value) {
     if (value instanceof Calendar) {
       return toShape((Calendar) value);
     } else if (value instanceof Date) {
@@ -227,7 +242,7 @@ public class DateRangePrefixTree extends
 
   /** Converts the Calendar into a Shape.
    * The isSet() state of the Calendar is re-instated when done. */
-  public Shape toShape(Calendar cal) {
+  public UnitNRShape toShape(Calendar cal) {
     // Convert a Calendar into a stack of cell numbers
     final int calPrecField = getCalPrecisionField(cal);//must call first; getters set all fields
     try {
@@ -256,19 +271,21 @@ public class DateRangePrefixTree extends
     }
   }
 
-  public Calendar toCalendar(Shape shape) {
-    if (shape instanceof LevelledValue)
-      return toCalendarLV((LevelledValue) shape);
-    throw new IllegalArgumentException("Can't be converted to Calendar: "+shape);
+  /** Calls {@link #toCalendar(org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape)}. */
+  @Override
+  public Object toObject(UnitNRShape shape) {
+    return toCalendar(shape);
   }
 
-  private Calendar toCalendarLV(LevelledValue lv) {
+  /** Converts the {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape} shape to a
+   * corresponding Calendar that is cleared below its level. */
+  public Calendar toCalendar(UnitNRShape lv) {
     if (lv.getLevel() == 0)
       return newCal();
-    if (comparePrefixLV(lv, minLV) <= 0) {//shouldn't typically happen; sometimes in a debugger
+    if (comparePrefix(lv, minLV) <= 0) {//shouldn't typically happen; sometimes in a debugger
       return (Calendar) MINCAL.clone();//full precision; truncation would cause underflow
     }
-    assert comparePrefixLV(lv, maxLV) <= 0;
+    assert comparePrefix(lv, maxLV) <= 0;
     Calendar cal = newCal();
 
     int yearAdj = lv.getValAtLevel(1) * 1_000_000;
@@ -294,23 +311,23 @@ public class DateRangePrefixTree extends
   }
 
   @Override
-  protected String toStringLV(LevelledValue lv) {
-    return toString(toCalendarLV(lv));
+  protected String toString(UnitNRShape lv) {
+    return toString(toCalendar(lv));
   }
 
   /** Calendar utility method:
-   * Converts to calendar to ISO-8601, to include proper BC handling (1BC is "0000", 2BC is "-0001", etc.);
+   * Formats the calendar to ISO-8601 format, to include proper BC handling (1BC is "0000", 2BC is "-0001", etc.);
    * and WITHOUT a trailing 'Z'.
    * A fully cleared calendar will yield the string "*".
    * The isSet() state of the Calendar is re-instated when done. */
    @SuppressWarnings("fallthrough")
-   public String toString(Calendar cal) {
+  public String toString(Calendar cal) {
     final int calPrecField = getCalPrecisionField(cal);//must call first; getters set all fields
     if (calPrecField == -1)
       return "*";
     try {
-      //TODO not fully optimized because I only expect this to be used in tests / debugging.
-      //  Borrow code from Solr DateUtil, and have it reference this back?
+      //TODO not fully optimized; but it's at least not used in 'search'.
+      //TODO maybe borrow code from Solr DateUtil (put in Lucene util somewhere), and have it reference this back?
       String pattern = "yyyy-MM-dd'T'HH:mm:ss.SSS";
       int ptnLen = 0;
       switch (calPrecField) {//switch fall-through is deliberate
@@ -352,8 +369,8 @@ public class DateRangePrefixTree extends
   }
 
   @Override
-  protected LevelledValue parseShapeLV(String str) throws ParseException {
-    return (LevelledValue) toShape(parseCalendar(str));
+  protected UnitNRShape parseUnitShape(String str) throws ParseException {
+    return toShape(parseCalendar(str));
   }
 
   /** Calendar utility method:

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java Sun Dec 14 22:50:20 2014
@@ -17,6 +17,8 @@ package org.apache.lucene.spatial.prefix
  * limitations under the License.
  */
 
+import java.text.ParseException;
+
 import com.spatial4j.core.context.SpatialContext;
 import com.spatial4j.core.context.SpatialContextFactory;
 import com.spatial4j.core.shape.Point;
@@ -27,11 +29,28 @@ import com.spatial4j.core.shape.impl.Rec
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.StringHelper;
 
-import java.text.ParseException;
-
 /**
- * A special SpatialPrefixTree for single-dimensional number ranges of integral values. It's based
- * on a stack of integers, and thus it's not limited to a long.
+ * A SpatialPrefixTree for single-dimensional numbers and number ranges of fixed precision values (not floating point).
+ * Despite its name, the indexed values (and queries) need not actually be ranges, they can be unit instance/values.
+ * <p />
+ * Why might you use this instead of Lucene's built-in integer/long support?  Here are some reasons with features based
+ * on code in this class, <em>or are possible based on this class but require a subclass to fully realize it</em>.
+ * <ul>
+ *   <li>Index ranges, not just unit instances. This is especially useful when the requirement calls for a
+ *   multi-valued range.</li>
+ *   <li>Instead of a fixed "precisionStep", this prefixTree can have a customizable number of child values for any
+ *   prefix (up to 32768). This allows exact alignment of the prefix-tree with typical/expected values, which
+ *   results in better performance.  For example in a Date implementation, every month can get its own dedicated prefix,
+ *   every day, etc., even though months vary in duration.</li>
+ *   <li>Arbitrary precision, like {@link java.math.BigDecimal}.</li>
+ *   <li>Standard Lucene integer/long indexing always indexes the full precision of those data types but this one
+ *   is customizable.</li>
+ * </ul>
+ *
+ * Unlike "normal" spatial components in this module, this special-purpose one only works with {@link Shape}s
+ * created by the methods on this class, not from any {@link com.spatial4j.core.context.SpatialContext}.
+ *
+ * @see org.apache.lucene.spatial.NumberRangePrefixTreeStrategy
  * @see <a href="https://issues.apache.org/jira/browse/LUCENE-5648">LUCENE-5648</a>
  * @lucene.experimental
  */
@@ -49,34 +68,93 @@ public abstract class NumberRangePrefixT
     DUMMY_CTX = factory.newSpatialContext();
   }
 
+  /** Base interface for {@link Shape}s this prefix tree supports. It extends {@link Shape} (Spatial4j) for compatibility
+   * with the spatial API even though it doesn't intermix with conventional 2D shapes.
+   * @lucene.experimental
+   */
+  public static interface NRShape extends Shape, Cloneable {
+    /** The result should be parseable by {@link #parseShape(String)}. */
+    abstract String toString();
+
+    /** Returns this shape rounded to the target level. If we are already more course than the level then the shape is
+     * simply returned.  The result may refer to internal state of the argument so you may want to clone it.
+     */
+    public NRShape roundToLevel(int targetLevel);
+  }
+
   //
-  //    LevelledValue
+  //  Factory / Conversions / parsing relating to NRShapes
   //
 
-  /** A value implemented as a stack of numbers. Spatially speaking, it's
-   * analogous to a Point but 1D yet has some precision width.
-   * @lucene.internal */
-  protected static interface LevelledValue extends Shape {
-    int getLevel();//0 means the world (universe).
-    int getValAtLevel(int level);//level >= 0 && <= getLevel()
-    LevelledValue getLVAtLevel(int level);
+  /** Converts the value to a unit shape. Doesn't parse strings; see {@link #parseShape(String)} for
+   * that. This is the reverse of {@link #toObject(org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape)}. */
+  public abstract UnitNRShape toUnitShape(Object value);
+
+  /** Returns a shape that represents the continuous range between {@code start} and {@code end}. It will
+   * be normalized, and so sometimes a {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape}
+   * will be returned, other times a
+   * {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.SpanUnitsNRShape} will be.
+   *
+   * @throws IllegalArgumentException if the arguments are in the wrong order, or if either contains the other (yet they
+   * aren't equal).
+   */
+  public NRShape toRangeShape(UnitNRShape startUnit, UnitNRShape endUnit) {
+    //note: this normalization/optimization process is actually REQUIRED based on assumptions elsewhere.
+    //Normalize start & end
+    startUnit = startUnit.getShapeAtLevel(truncateStartVals(startUnit, 0)); // chops off trailing min-vals (zeroes)
+    endUnit = endUnit.getShapeAtLevel(truncateEndVals(endUnit, 0)); // chops off trailing max-vals
+    //Optimize to just start or end if it's equivalent, e.g. April to April 1st is April 1st.
+    int cmp = comparePrefix(startUnit, endUnit);
+    if (cmp > 0) {
+      throw new IllegalArgumentException("Wrong order: "+ startUnit +" TO "+ endUnit);
+    }
+    if (cmp == 0) {//one is a prefix of the other
+      if (startUnit.getLevel() == endUnit.getLevel()) {
+        //same
+        return startUnit;
+      } else if (endUnit.getLevel() > startUnit.getLevel()) {
+        // e.g. April to April 1st
+        if (truncateStartVals(endUnit, startUnit.getLevel()) == startUnit.getLevel()) {
+          return endUnit;
+        }
+      } else {//minLV level > maxLV level
+        // e.g. April 30 to April
+        if (truncateEndVals(startUnit, endUnit.getLevel()) == endUnit.getLevel()) {
+          return startUnit;
+        }
+      }
+    }
+    return new SpanUnitsNRShape(startUnit, endUnit);
   }
 
-  /** Compares a to b, returning less than 0, 0, or greater than 0, if a is less than, equal to, or
-   * greater than b, respectively. Only min(a.levels,b.levels) are compared.
-   * @lucene.internal */
-  protected static int comparePrefixLV(LevelledValue a, LevelledValue b) {
-    int minLevel = Math.min(a.getLevel(), b.getLevel());
-    for (int level = 1; level <= minLevel; level++) {
-      int diff = a.getValAtLevel(level) - b.getValAtLevel(level);
-      if (diff != 0)
-        return diff;
+  /** From lv.getLevel on up, it returns the first Level seen with val != 0. It doesn't check past endLevel. */
+  private int truncateStartVals(UnitNRShape lv, int endLevel) {
+    for (int level = lv.getLevel(); level > endLevel; level--) {
+      if (lv.getValAtLevel(level) != 0)
+        return level;
     }
-    return 0;
+    return endLevel;
   }
 
-  protected String toStringLV(LevelledValue lv) {
-    StringBuilder buf = new StringBuilder();
+  private int truncateEndVals(UnitNRShape lv, int endLevel) {
+    for (int level = lv.getLevel(); level > endLevel; level--) {
+      int max = getNumSubCells(lv.getShapeAtLevel(level - 1)) - 1;
+      if (lv.getValAtLevel(level) != max)
+        return level;
+    }
+    return endLevel;
+  }
+
+  /** Converts a UnitNRShape shape to the corresponding type supported by this class, such as a Calendar/BigDecimal.
+   * This is the reverse of {@link #toUnitShape(Object)}.
+   */
+  public abstract Object toObject(UnitNRShape shape);
+
+  /** A string representation of the UnitNRShape that is parse-able by {@link #parseUnitShape(String)}. */
+  protected abstract String toString(UnitNRShape lv);
+
+  protected static String toStringUnitRaw(UnitNRShape lv) {
+    StringBuilder buf = new StringBuilder(100);
     buf.append('[');
     for (int level = 1; level <= lv.getLevel(); level++) {
       buf.append(lv.getValAtLevel(level)).append(',');
@@ -86,20 +164,96 @@ public abstract class NumberRangePrefixT
     return buf.toString();
   }
 
+  /** Detects a range pattern and parses it, otherwise it's parsed as one shape via
+   * {@link #parseUnitShape(String)}.  The range pattern looks like this BNF:
+   * <pre>
+   *   '[' + parseShapeLV + ' TO ' + parseShapeLV + ']'
+   * </pre>
+   * It's the same thing as the toString() of the range shape, notwithstanding range optimization.
+   *
+   * @param str not null or empty
+   * @return not null
+   * @throws java.text.ParseException If there is a problem
+   */
+  public NRShape parseShape(String str) throws ParseException {
+    if (str == null || str.isEmpty())
+      throw new IllegalArgumentException("str is null or blank");
+    if (str.charAt(0) == '[') {
+      if (str.charAt(str.length()-1) != ']')
+        throw new ParseException("If starts with [ must end with ]; got "+str, str.length()-1);
+      int middle = str.indexOf(" TO ");
+      if (middle < 0)
+        throw new ParseException("If starts with [ must contain ' TO '; got "+str, -1);
+      String leftStr = str.substring(1, middle);
+      String rightStr = str.substring(middle + " TO ".length(), str.length()-1);
+      return toRangeShape(parseUnitShape(leftStr), parseUnitShape(rightStr));
+    } else if (str.charAt(0) == '{') {
+      throw new ParseException("Exclusive ranges not supported; got "+str, 0);
+    } else {
+      return parseUnitShape(str);
+    }
+  }
+
+  /** Parse a String to a UnitNRShape. "*" should be the full-range (level 0 shape). */
+  protected abstract UnitNRShape parseUnitShape(String str) throws ParseException;
+
+
   //
-  //    NRShape
+  //    UnitNRShape
   //
 
-  /** Number Range Shape; based on a pair of {@link LevelledValue}.
-   * Spatially speaking, it's analogous to a Rectangle but 1D.
+  /**
+   * A unit value Shape implemented as a stack of numbers, one for each level in the prefix tree. It directly
+   * corresponds to a {@link Cell}.  Spatially speaking, it's analogous to a Point but 1D and has some precision width.
+   * @lucene.experimental
+   */
+  public static interface UnitNRShape extends NRShape, Comparable<UnitNRShape> {
+    //note: formerly known as LevelledValue; thus some variables still use 'lv'
+
+    /** Get the prefix tree level, the higher the more precise. 0 means the world (universe). */
+    int getLevel();
+    /** Gets the value at the specified level of this unit. level must be &gt;= 0 and &lt;= getLevel(). */
+    int getValAtLevel(int level);
+    /** Gets an ancestor at the specified level. It shares state, so you may want to clone() it. */
+    UnitNRShape getShapeAtLevel(int level);
+    @Override
+    UnitNRShape roundToLevel(int targetLevel);
+
+    /** Deep clone */
+    UnitNRShape clone();
+  }
+
+  /** Compares a to b, returning less than 0, 0, or greater than 0, if a is less than, equal to, or
+   * greater than b, respectively, up to their common prefix (i.e. only min(a.levels,b.levels) are compared).
    * @lucene.internal */
-  protected class NRShape implements Shape {
+  protected static int comparePrefix(UnitNRShape a, UnitNRShape b) {
+    int minLevel = Math.min(a.getLevel(), b.getLevel());
+    for (int level = 1; level <= minLevel; level++) {
+      int diff = a.getValAtLevel(level) - b.getValAtLevel(level);
+      if (diff != 0)
+        return diff;
+    }
+    return 0;
+  }
+
+
+  //
+  //    SpanUnitsNRShape
+  //
 
-    private final LevelledValue minLV, maxLV;
+  /** A range Shape; based on a pair of {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape}.
+   * Spatially speaking, it's analogous to a Rectangle but 1D. It might have been named with Range in the name but it
+   * may be confusing since even the {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape}
+   * is in some sense a range.
+   * @lucene.experimental */
+  public class SpanUnitsNRShape implements NRShape {
+
+    private final UnitNRShape minLV, maxLV;
     private final int lastLevelInCommon;//computed; not part of identity
 
-    /** Don't call directly; see {@link #toRangeShape(com.spatial4j.core.shape.Shape, com.spatial4j.core.shape.Shape)}. */
-    private NRShape(LevelledValue minLV, LevelledValue maxLV) {
+    /** Don't call directly; see
+     * {@link #toRangeShape(org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape, org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape)}. */
+    private SpanUnitsNRShape(UnitNRShape minLV, UnitNRShape maxLV) {
       this.minLV = minLV;
       this.maxLV = maxLV;
 
@@ -112,37 +266,42 @@ public abstract class NumberRangePrefixT
       lastLevelInCommon = level - 1;
     }
 
-    public LevelledValue getMinLV() { return minLV; }
+    public UnitNRShape getMinUnit() { return minLV; }
+
+    public UnitNRShape getMaxUnit() { return maxLV; }
 
-    public LevelledValue getMaxLV() { return maxLV; }
+    /** How many levels are in common between minUnit and maxUnit, not including level 0. */
+    private int getLevelsInCommon() { return lastLevelInCommon; }
 
-    /** How many levels are in common between minLV and maxLV. */
-    private int getLastLevelInCommon() { return lastLevelInCommon; }
+    @Override
+    public NRShape roundToLevel(int targetLevel) {
+      return toRangeShape(minLV.roundToLevel(targetLevel), maxLV.roundToLevel(targetLevel));
+    }
 
     @Override
     public SpatialRelation relate(Shape shape) {
-//      if (shape instanceof LevelledValue)
-//        return relate((LevelledValue)shape);
-      if (shape instanceof NRShape)
-        return relate((NRShape) shape);
-      return shape.relate(this).transpose();//probably a LevelledValue
+//      if (shape instanceof UnitNRShape)
+//        return relate((UnitNRShape)shape);
+      if (shape instanceof SpanUnitsNRShape)
+        return relate((SpanUnitsNRShape) shape);
+      return shape.relate(this).transpose();//probably a UnitNRShape
     }
 
-    public SpatialRelation relate(NRShape ext) {
+    public SpatialRelation relate(SpanUnitsNRShape ext) {
       //This logic somewhat mirrors RectangleImpl.relate_range()
-      int extMin_intMax = comparePrefixLV(ext.getMinLV(), getMaxLV());
+      int extMin_intMax = comparePrefix(ext.getMinUnit(), getMaxUnit());
       if (extMin_intMax > 0)
         return SpatialRelation.DISJOINT;
-      int extMax_intMin = comparePrefixLV(ext.getMaxLV(), getMinLV());
+      int extMax_intMin = comparePrefix(ext.getMaxUnit(), getMinUnit());
       if (extMax_intMin < 0)
         return SpatialRelation.DISJOINT;
-      int extMin_intMin = comparePrefixLV(ext.getMinLV(), getMinLV());
-      int extMax_intMax = comparePrefixLV(ext.getMaxLV(), getMaxLV());
-      if ((extMin_intMin > 0 || extMin_intMin == 0 && ext.getMinLV().getLevel() >= getMinLV().getLevel())
-          && (extMax_intMax < 0 || extMax_intMax == 0 && ext.getMaxLV().getLevel() >= getMaxLV().getLevel()))
+      int extMin_intMin = comparePrefix(ext.getMinUnit(), getMinUnit());
+      int extMax_intMax = comparePrefix(ext.getMaxUnit(), getMaxUnit());
+      if ((extMin_intMin > 0 || extMin_intMin == 0 && ext.getMinUnit().getLevel() >= getMinUnit().getLevel())
+          && (extMax_intMax < 0 || extMax_intMax == 0 && ext.getMaxUnit().getLevel() >= getMaxUnit().getLevel()))
         return SpatialRelation.CONTAINS;
-      if ((extMin_intMin < 0 || extMin_intMin == 0 && ext.getMinLV().getLevel() <= getMinLV().getLevel())
-          && (extMax_intMax > 0 || extMax_intMax == 0 && ext.getMaxLV().getLevel() <= getMaxLV().getLevel()))
+      if ((extMin_intMin < 0 || extMin_intMin == 0 && ext.getMinUnit().getLevel() <= getMinUnit().getLevel())
+          && (extMax_intMax > 0 || extMax_intMax == 0 && ext.getMaxUnit().getLevel() <= getMaxUnit().getLevel()))
         return SpatialRelation.WITHIN;
       return SpatialRelation.INTERSECTS;
     }
@@ -165,18 +324,27 @@ public abstract class NumberRangePrefixT
     @Override
     public boolean isEmpty() { return false; }
 
+    /** A deep clone. */
+    @Override
+    public SpanUnitsNRShape clone() {
+      return new SpanUnitsNRShape(minLV.clone(), maxLV.clone());
+    }
+
     @Override
-    public String toString() { return "[" + toStringLV(minLV) + " TO " + toStringLV(maxLV) + "]"; }
+    public String toString() {
+      return "[" + NumberRangePrefixTree.this.toString(minLV) + " TO "
+          + NumberRangePrefixTree.this.toString(maxLV) + "]";
+    }
 
     @Override
     public boolean equals(Object o) {
       if (this == o) return true;
       if (o == null || getClass() != o.getClass()) return false;
 
-      NRShape nrShape = (NRShape) o;
+      SpanUnitsNRShape spanShape = (SpanUnitsNRShape) o;
 
-      if (!maxLV.equals(nrShape.maxLV)) return false;
-      if (!minLV.equals(nrShape.minLV)) return false;
+      if (!maxLV.equals(spanShape.maxLV)) return false;
+      if (!minLV.equals(spanShape.minLV)) return false;
 
       return true;
     }
@@ -187,98 +355,7 @@ public abstract class NumberRangePrefixT
       result = 31 * result + maxLV.hashCode();
       return result;
     }
-  }// class NRShapeImpl
-
-  /** Converts the value to a shape (usually not a range). If it's a JDK object (e.g. Number, Calendar)
-   * that could be parsed from a String, this class won't do it; you must parse it. */
-  public abstract Shape toShape(Object value);
-
-  /** Detects a range pattern and parses it, otherwise it's parsed as one shape via
-   * {@link #parseShapeLV(String)}.  The range pattern looks like this BNF:
-   * <pre>
-   *   '[' + parseShapeLV + ' TO ' + parseShapeLV + ']'
-   * </pre>
-   * It's the same thing as the toString() of the range shape, notwithstanding range optimization.
-   * @param str not null or empty
-   * @return not null
-   * @throws java.text.ParseException If there is a problem
-   */
-  public Shape parseShape(String str) throws ParseException {
-    if (str == null || str.isEmpty())
-      throw new IllegalArgumentException("str is null or blank");
-    if (str.charAt(0) == '[') {
-      if (str.charAt(str.length()-1) != ']')
-        throw new ParseException("If starts with [ must end with ]; got "+str, str.length()-1);
-      int middle = str.indexOf(" TO ");
-      if (middle < 0)
-        throw new ParseException("If starts with [ must contain ' TO '; got "+str, -1);
-      String leftStr = str.substring(1, middle);
-      String rightStr = str.substring(middle + " TO ".length(), str.length()-1);
-      return toRangeShape(parseShapeLV(leftStr), parseShapeLV(rightStr));
-    } else if (str.charAt(0) == '{') {
-      throw new ParseException("Exclusive ranges not supported; got "+str, 0);
-    } else {
-      return parseShapeLV(str);
-    }
-  }
-
-  /** Parse a String to a LevelledValue. "*" should be the full-range. */
-  protected abstract LevelledValue parseShapeLV(String str) throws ParseException;
-
-  /** Returns a shape that represents the continuous range between {@code start} and {@code end}. It will
-   * be optimized.
-   * @throws IllegalArgumentException if the arguments are in the wrong order, or if either contains the other.
-   */
-  public Shape toRangeShape(Shape start, Shape end) {
-    if (!(start instanceof LevelledValue && end instanceof LevelledValue))
-      throw new IllegalArgumentException("Must pass "+LevelledValue.class+" but got "+start.getClass());
-    LevelledValue startLV = (LevelledValue) start;
-    LevelledValue endLV = (LevelledValue) end;
-    //note: this normalization/optimization process is actually REQUIRED based on assumptions elsewhere.
-    //Normalize start & end
-    startLV = startLV.getLVAtLevel(truncateStartVals(startLV, 0)); // chops off trailing min-vals (zeroes)
-    endLV = endLV.getLVAtLevel(truncateEndVals(endLV, 0)); // chops off trailing max-vals
-    //Optimize to just start or end if it's equivalent, e.g. April to April 1st is April 1st.
-    int cmp = comparePrefixLV(startLV, endLV);
-    if (cmp > 0) {
-      throw new IllegalArgumentException("Wrong order: "+start+" TO "+end);
-    }
-    if (cmp == 0) {//one is a prefix of the other
-      if (startLV.getLevel() == endLV.getLevel()) {
-        //same
-        return startLV;
-      } else if (endLV.getLevel() > startLV.getLevel()) {
-        // e.g. April to April 1st
-        if (truncateStartVals(endLV, startLV.getLevel()) == startLV.getLevel()) {
-          return endLV;
-  }
-      } else {//minLV level > maxLV level
-        // e.g. April 30 to April
-        if (truncateEndVals(startLV, endLV.getLevel()) == endLV.getLevel()) {
-          return startLV;
-        }
-      }
-    }
-    return new NRShape(startLV, endLV);
-  }
-
-  /** From lv.getLevel on up, it returns the first Level seen with val != 0. It doesn't check past endLevel. */
-  private int truncateStartVals(LevelledValue lv, int endLevel) {
-    for (int level = lv.getLevel(); level > endLevel; level--) {
-      if (lv.getValAtLevel(level) != 0)
-        return level;
-    }
-    return endLevel;
-  }
-
-  private int truncateEndVals(LevelledValue lv, int endLevel) {
-    for (int level = lv.getLevel(); level > endLevel; level--) {
-      int max = getNumSubCells(lv.getLVAtLevel(level-1)) - 1;
-      if (lv.getValAtLevel(level) != max)
-        return level;
-    }
-    return endLevel;
-  }
+  }// class SpanUnitsNRShape
 
   //
   //    NumberRangePrefixTree
@@ -332,15 +409,19 @@ public abstract class NumberRangePrefixT
 
   @Override
   public int getLevelForDistance(double dist) {
-    return maxLevels;
+    //note: it might be useful to compute which level has a raw width (counted in
+    // bottom units, e.g. milliseconds), that covers the provided dist in those units?
+    return maxLevels; // thus always use full precision. We don't do approximations in this tree/strategy.
+    //throw new UnsupportedOperationException("Not applicable.");
   }
 
   @Override
   public double getDistanceForLevel(int level) {
+    //note: we could compute this... should we?
     throw new UnsupportedOperationException("Not applicable.");
   }
 
-  protected Shape toShape(int[] valStack, int len) {
+  protected UnitNRShape toShape(int[] valStack, int len) {
     final NRCell[] cellStack = newCellStack(len);
     for (int i = 0; i < len; i++) {
       cellStack[i+1].resetCellWithCellNum(valStack[i]);
@@ -367,7 +448,7 @@ public abstract class NumberRangePrefixT
     if (scratch == null)
       scratch = getWorldCell();
 
-    //We decode level, leaf, and populate bytes.
+    //We decode level #, leaf boolean, and populate bytes by reference. We don't decode the stack.
 
     //reverse lookup term length to the level and hence the cell
     NRCell[] cellsByLevel = ((NRCell) scratch).cellsByLevel;
@@ -389,7 +470,8 @@ public abstract class NumberRangePrefixT
     return result;
   }
 
-  protected int getNumSubCells(LevelledValue lv) {
+  /** Returns the number of sub-cells beneath the given UnitNRShape. */
+  public int getNumSubCells(UnitNRShape lv) {
     return maxSubCellsByLevel[lv.getLevel()];
   }
 
@@ -402,7 +484,7 @@ public abstract class NumberRangePrefixT
    * of Cells at adjacent levels, that all have a reference back to the cell array to traverse. They also share a common
    * BytesRef for the term.
    * @lucene.internal */
-  protected class NRCell extends CellIterator implements Cell, LevelledValue {
+  protected class NRCell extends CellIterator implements Cell, UnitNRShape {
 
     //Shared: (TODO put this in a new class)
     final NRCell[] cellsByLevel;
@@ -484,11 +566,15 @@ public abstract class NumberRangePrefixT
           cell.cellNumber = (term.bytes[term.offset + termLen - 1] & 0xFF) - 1;
           assert cell.cellNumber < 255;
         }
-        assert cell.cellNumber >= 0;
+        cell.assertDecoded();
       }
     }
 
-    @Override // for Cell & for LevelledValue
+    private void assertDecoded() {
+      assert cellNumber >= 0 : "Illegal state; ensureDecoded() wasn't called";
+    }
+
+    @Override // for Cell & for UnitNRShape
     public int getLevel() {
       return cellLevel;
     }
@@ -514,8 +600,9 @@ public abstract class NumberRangePrefixT
     }
 
     @Override
-    public Shape getShape() {
-      ensureDecoded(); return this;
+    public UnitNRShape getShape() {
+      ensureDecoded();
+      return this;
     }
 
     @Override
@@ -579,7 +666,7 @@ public abstract class NumberRangePrefixT
 
     //----------- CellIterator
 
-    Shape iterFilter;//LevelledValue or NRShape
+    Shape iterFilter;//UnitNRShape or NRShape
     boolean iterFirstIsIntersects;
     boolean iterLastIsIntersects;
     int iterFirstCellNumber;
@@ -587,11 +674,11 @@ public abstract class NumberRangePrefixT
 
     private void initIter(Shape filter) {
       cellNumber = -1;
-      if (filter instanceof LevelledValue && ((LevelledValue) filter).getLevel() == 0)
+      if (filter instanceof UnitNRShape && ((UnitNRShape) filter).getLevel() == 0)
         filter = null;//world means everything -- no filter
       iterFilter = filter;
 
-      NRCell parent = getLVAtLevel(getLevel() - 1);
+      NRCell parent = getShapeAtLevel(getLevel() - 1);
 
       // Initialize iter* members.
 
@@ -604,16 +691,16 @@ public abstract class NumberRangePrefixT
         return;
       }
 
-      final LevelledValue minLV;
-      final LevelledValue maxLV;
+      final UnitNRShape minLV;
+      final UnitNRShape maxLV;
       final int lastLevelInCommon;//between minLV & maxLV
-      if (filter instanceof NRShape) {
-        NRShape nrShape = (NRShape) iterFilter;
-        minLV = nrShape.getMinLV();
-        maxLV = nrShape.getMaxLV();
-        lastLevelInCommon = nrShape.getLastLevelInCommon();
+      if (filter instanceof SpanUnitsNRShape) {
+        SpanUnitsNRShape spanShape = (SpanUnitsNRShape) iterFilter;
+        minLV = spanShape.getMinUnit();
+        maxLV = spanShape.getMaxUnit();
+        lastLevelInCommon = spanShape.getLevelsInCommon();
       } else {
-        minLV = (LevelledValue) iterFilter;
+        minLV = (UnitNRShape) iterFilter;
         maxLV = minLV;
         lastLevelInCommon = minLV.getLevel();
       }
@@ -649,7 +736,7 @@ public abstract class NumberRangePrefixT
 
       //not common to get here, except for level 1 which always happens
 
-      int startCmp = comparePrefixLV(minLV, parent);
+      int startCmp = comparePrefix(minLV, parent);
       if (startCmp > 0) {//start comes after this cell
         iterFirstCellNumber = 0;
         iterFirstIsIntersects = false;
@@ -657,7 +744,7 @@ public abstract class NumberRangePrefixT
         iterLastIsIntersects = false;
         return;
       }
-      int endCmp = comparePrefixLV(maxLV, parent);//compare to end cell
+      int endCmp = comparePrefix(maxLV, parent);//compare to end cell
       if (endCmp < 0) {//end comes before this cell
         iterFirstCellNumber = 0;
         iterFirstIsIntersects = false;
@@ -719,56 +806,65 @@ public abstract class NumberRangePrefixT
 
     //TODO override nextFrom to be more efficient
 
-    //----------- LevelledValue / Shape
+    //----------- UnitNRShape
 
     @Override
     public int getValAtLevel(int level) {
       final int result = cellsByLevel[level].cellNumber;
-      assert result >= 0;//initialized
+      assert result >= 0;//initialized (decoded)
       return result;
     }
 
     @Override
-    public NRCell getLVAtLevel(int level) {
+    public NRCell getShapeAtLevel(int level) {
       assert level <= cellLevel;
       return cellsByLevel[level];
     }
 
     @Override
+    public UnitNRShape roundToLevel(int targetLevel) {
+      if (getLevel() <= targetLevel) {
+        return this;
+      } else {
+        return getShapeAtLevel(targetLevel);
+      }
+    }
+
+    @Override
     public SpatialRelation relate(Shape shape) {
-      ensureDecoded();
+      assertDecoded();
       if (shape == iterFilter && cellShapeRel != null)
         return cellShapeRel;
-      if (shape instanceof LevelledValue)
-        return relate((LevelledValue)shape);
-      if (shape instanceof NRShape)
-        return relate((NRShape)shape);
+      if (shape instanceof UnitNRShape)
+        return relate((UnitNRShape)shape);
+      if (shape instanceof SpanUnitsNRShape)
+        return relate((SpanUnitsNRShape)shape);
       return shape.relate(this).transpose();
     }
 
-    public SpatialRelation relate(LevelledValue lv) {
-      ensureDecoded();
-      int cmp = comparePrefixLV(this, lv);
+    public SpatialRelation relate(UnitNRShape lv) {
+      assertDecoded();
+      int cmp = comparePrefix(this, lv);
       if (cmp != 0)
         return SpatialRelation.DISJOINT;
       if (getLevel() > lv.getLevel())
-        return SpatialRelation.WITHIN;//or equals
-      return SpatialRelation.CONTAINS;
+        return SpatialRelation.WITHIN;
+      return SpatialRelation.CONTAINS;//or equals
       //no INTERSECTS; that won't happen.
     }
 
-    public SpatialRelation relate(NRShape nrShape) {
-      ensureDecoded();
-      int startCmp = comparePrefixLV(nrShape.getMinLV(), this);
+    public SpatialRelation relate(SpanUnitsNRShape spanShape) {
+      assertDecoded();
+      int startCmp = comparePrefix(spanShape.getMinUnit(), this);
       if (startCmp > 0) {//start comes after this cell
         return SpatialRelation.DISJOINT;
       }
-      int endCmp = comparePrefixLV(nrShape.getMaxLV(), this);
+      int endCmp = comparePrefix(spanShape.getMaxUnit(), this);
       if (endCmp < 0) {//end comes before this cell
         return SpatialRelation.DISJOINT;
       }
-      int nrMinLevel = nrShape.getMinLV().getLevel();
-      int nrMaxLevel = nrShape.getMaxLV().getLevel();
+      int nrMinLevel = spanShape.getMinUnit().getLevel();
+      int nrMaxLevel = spanShape.getMaxUnit().getLevel();
       if ((startCmp < 0 || startCmp == 0 && nrMinLevel <= getLevel())
           && (endCmp > 0 || endCmp == 0 && nrMaxLevel <= getLevel()))
         return SpatialRelation.WITHIN;//or equals
@@ -781,13 +877,33 @@ public abstract class NumberRangePrefixT
           return SpatialRelation.INTERSECTS;
       }
       for (;nrMaxLevel < getLevel(); nrMaxLevel++) {
-        if (getValAtLevel(nrMaxLevel + 1) != getNumSubCells(getLVAtLevel(nrMaxLevel)) - 1)
+        if (getValAtLevel(nrMaxLevel + 1) != getNumSubCells(getShapeAtLevel(nrMaxLevel)) - 1)
           return SpatialRelation.INTERSECTS;
       }
       return SpatialRelation.CONTAINS;
     }
 
     @Override
+    public UnitNRShape clone() {
+      //no leaf distinction; this is purely based on UnitNRShape
+      NRCell cell = (NRCell) readCell(getTokenBytesNoLeaf(null), null);
+      cell.ensureOwnTermBytes();
+      return cell.getShape();
+    }
+
+    @Override
+    public int compareTo(UnitNRShape o) {
+      assertDecoded();
+      //no leaf distinction; this is purely based on UnitNRShape
+      int cmp = comparePrefix(this, o);
+      if (cmp != 0) {
+        return cmp;
+      } else {
+        return getLevel() - o.getLevel();
+      }
+    }
+
+    @Override
     public Rectangle getBoundingBox() {
       throw new UnsupportedOperationException();
     }
@@ -848,11 +964,7 @@ public abstract class NumberRangePrefixT
 
     @Override
     public String toString() {
-      ensureDecoded();
-      String str = NumberRangePrefixTree.this.toStringLV(this);
-      if (isLeaf())
-        str += "•";//bullet (won't be confused with textual representation)
-      return str;
+      return NumberRangePrefixTree.this.toString(getShape());
     }
 
     /** Configure your IDE to use this. */
@@ -860,16 +972,7 @@ public abstract class NumberRangePrefixT
       String pretty = toString();
       if (getLevel() == 0)
         return pretty;
-      //now prefix it by an array of integers of the cell levels
-      StringBuilder buf = new StringBuilder(100);
-      buf.append('[');
-      for (int level = 1; level <= getLevel(); level++) {
-        if (level != 1)
-          buf.append(',');
-        buf.append(getLVAtLevel(level).cellNumber);
-      }
-      buf.append("] ").append(pretty);
-      return buf.toString();
+      return toStringUnitRaw(this) + (isLeaf() ? "•" : "") + " " + pretty;
     }
 
   } // END OF NRCell

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java Sun Dec 14 22:50:20 2014
@@ -17,18 +17,19 @@ package org.apache.lucene.spatial.prefix
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Calendar;
+
 import com.carrotsearch.randomizedtesting.annotations.Repeat;
 import com.spatial4j.core.shape.Shape;
 import org.apache.lucene.spatial.NumberRangePrefixTreeStrategy;
 import org.apache.lucene.spatial.prefix.tree.DateRangePrefixTree;
+import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape;
 import org.apache.lucene.spatial.query.SpatialOperation;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.text.ParseException;
-import java.util.Calendar;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
 
 public class DateNRStrategyTest extends RandomSpatialOpStrategyTestCase {
 
@@ -36,16 +37,17 @@ public class DateNRStrategyTest extends
 
   DateRangePrefixTree tree;
 
-  int era;
-  int year;
+  long randomCalWindowMs;
 
   @Before
   public void setUp() throws Exception {
     super.setUp();
     tree = DateRangePrefixTree.INSTANCE;
     strategy = new NumberRangePrefixTreeStrategy(tree, "dateRange");
-    era = random().nextBoolean() ? 0 : 1;
-    year = 1 + random().nextInt(2_000_000);
+    Calendar tmpCal = tree.newCal();
+    int randomCalWindowField = randomIntBetween(1, Calendar.ZONE_OFFSET - 1);//we're not allowed to add zone offset
+    tmpCal.add(randomCalWindowField, 2_000);
+    randomCalWindowMs = Math.max(2000L, tmpCal.getTimeInMillis());
   }
 
   @Test
@@ -69,9 +71,6 @@ public class DateNRStrategyTest extends
   @Test
   public void testWithinSame() throws IOException {
     final Calendar cal = tree.newCal();
-    cal.set(Calendar.ERA, era);
-    cal.set(Calendar.YEAR, year);
-
     testOperation(
         tree.toShape(cal),
         SpatialOperation.IsWithin,
@@ -98,10 +97,13 @@ public class DateNRStrategyTest extends
   @Override
   protected Shape randomIndexedShape() {
     Calendar cal1 = randomCalendar();
-    Shape s1 = tree.toShape(cal1);
+    UnitNRShape s1 = tree.toShape(cal1);
+    if (rarely()) {
+      return s1;
+    }
     try {
       Calendar cal2 = randomCalendar();
-      Shape s2 = tree.toShape(cal2);
+      UnitNRShape s2 = tree.toShape(cal2);
       if (cal1.compareTo(cal2) < 0) {
         return tree.toRangeShape(s1, s2);
       } else {
@@ -115,9 +117,7 @@ public class DateNRStrategyTest extends
 
   private Calendar randomCalendar() {
     Calendar cal = tree.newCal();
-    cal.setTimeInMillis(random().nextLong());
-    cal.set(Calendar.ERA, era);
-    cal.set(Calendar.YEAR, year);
+    cal.setTimeInMillis(random().nextLong() % randomCalWindowMs);
     try {
       tree.clearFieldsAfter(cal, random().nextInt(Calendar.FIELD_COUNT+1)-1);
     } catch (AssertionError e) {

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTreeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTreeTest.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTreeTest.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTreeTest.java Sun Dec 14 22:50:20 2014
@@ -17,16 +17,17 @@ package org.apache.lucene.spatial.prefix
  * limitations under the License.
  */
 
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.LuceneTestCase;
-
 import java.text.ParseException;
 import java.util.Arrays;
 import java.util.Calendar;
 import java.util.GregorianCalendar;
 
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
 public class DateRangePrefixTreeTest extends LuceneTestCase {
 
   private DateRangePrefixTree tree = DateRangePrefixTree.INSTANCE;
@@ -94,7 +95,7 @@ public class DateRangePrefixTreeTest ext
       assertEquals(cal, tree.parseCalendar(calString));
 
       //to Shape and back to Cal
-      Shape shape = tree.toShape(cal);
+      UnitNRShape shape = tree.toShape(cal);
       Calendar cal2 = tree.toCalendar(shape);
       assertEquals(calString, tree.toString(cal2));
 
@@ -104,7 +105,7 @@ public class DateRangePrefixTreeTest ext
         BytesRef term = cell.getTokenBytesNoLeaf(null);
         Cell cell2 = tree.readCell(BytesRef.deepCopyOf(term), null);
         assertEquals(calString, cell, cell2);
-        Calendar cal3 = tree.toCalendar(cell2.getShape());
+        Calendar cal3 = tree.toCalendar((UnitNRShape) cell2.getShape());
         assertEquals(calString, tree.toString(cal3));
 
         // setLeaf comparison

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java Sun Dec 14 22:50:20 2014
@@ -29,8 +29,8 @@ public class HighCompressionCompressingC
 
   /** Default constructor. */
   public HighCompressionCompressingCodec() {
-    // no need to have a higher block length than 32KB since deflate splits
-    // into blocks of 32KB anyway, and this is a lower bound (try to avoid > 32KB)
-    this(24576, 512, false);
+    // we don't worry about zlib block overhead as its
+    // not bad and try to save space instead:
+    this(61440, 512, false);
   }
 }

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Sun Dec 14 22:50:20 2014
@@ -134,6 +134,16 @@ public class MockDirectoryWrapper extend
   public int getInputCloneCount() {
     return inputCloneCount.get();
   }
+  
+  boolean verboseClone;
+  
+  /** 
+   * If set to true, we print a fake exception
+   * with filename and stacktrace on every indexinput clone()
+   */
+  public void setVerboseClone(boolean v) {
+    verboseClone = v;
+  }
 
   public void setTrackDiskUsage(boolean v) {
     trackDiskUsage = v;

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexInputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexInputWrapper.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexInputWrapper.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexInputWrapper.java Sun Dec 14 22:50:20 2014
@@ -67,6 +67,9 @@ public class MockIndexInputWrapper exten
   @Override
   public MockIndexInputWrapper clone() {
     ensureOpen();
+    if (dir.verboseClone) {
+      new Exception("clone: " + this).printStackTrace(System.out);
+    }
     dir.inputCloneCount.incrementAndGet();
     IndexInput iiclone = delegate.clone();
     MockIndexInputWrapper clone = new MockIndexInputWrapper(dir, name, iiclone);
@@ -91,6 +94,9 @@ public class MockIndexInputWrapper exten
   @Override
   public IndexInput slice(String sliceDescription, long offset, long length) throws IOException {
     ensureOpen();
+    if (dir.verboseClone) {
+      new Exception("slice: " + this).printStackTrace(System.out);
+    }
     dir.inputCloneCount.incrementAndGet();
     IndexInput slice = delegate.slice(sliceDescription, offset, length);
     MockIndexInputWrapper clone = new MockIndexInputWrapper(dir, sliceDescription, slice);

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java Sun Dec 14 22:50:20 2014
@@ -22,15 +22,13 @@ import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.util.AbstractList;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.IdentityHashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.NoSuchElementException;
+import java.util.Set;
 
 /** Crawls object graph to collect RAM usage for testing */
 public final class RamUsageTester {
@@ -91,7 +89,7 @@ public final class RamUsageTester {
    */
   private static long measureObjectSize(Object root, Accumulator accumulator) {
     // Objects seen so far.
-    final IdentityHashSet<Object> seen = new IdentityHashSet<>();
+    final Set<Object> seen = Collections.newSetFromMap(new IdentityHashMap<Object, Boolean>());
     // Class cache with reference Field and precalculated shallow size. 
     final IdentityHashMap<Class<?>, ClassCache> classCache = new IdentityHashMap<>();
     // Stack of objects pending traversal. Recursion caused stack overflows. 
@@ -213,243 +211,5 @@ public final class RamUsageTester {
         referenceFields.toArray(new Field[referenceFields.size()]));
     return cachedInfo;
   }
-  
-  /**
-   * An identity hash set implemented using open addressing. No null keys are allowed.
-   * 
-   * TODO: If this is useful outside this class, make it public - needs some work
-   */
-  static final class IdentityHashSet<KType> implements Iterable<KType> {
-    /**
-     * Default load factor.
-     */
-    public final static float DEFAULT_LOAD_FACTOR = 0.75f;
-
-    /**
-     * Minimum capacity for the set.
-     */
-    public final static int MIN_CAPACITY = 4;
-
-    /**
-     * All of set entries. Always of power of two length.
-     */
-    public Object[] keys;
-    
-    /**
-     * Cached number of assigned slots.
-     */
-    public int assigned;
-    
-    /**
-     * The load factor for this set (fraction of allocated or deleted slots before
-     * the buffers must be rehashed or reallocated).
-     */
-    public final float loadFactor;
-    
-    /**
-     * Cached capacity threshold at which we must resize the buffers.
-     */
-    private int resizeThreshold;
-    
-    /**
-     * Creates a hash set with the default capacity of 16.
-     * load factor of {@value #DEFAULT_LOAD_FACTOR}. `
-     */
-    public IdentityHashSet() {
-      this(16, DEFAULT_LOAD_FACTOR);
-    }
-    
-    /**
-     * Creates a hash set with the given capacity, load factor of
-     * {@value #DEFAULT_LOAD_FACTOR}.
-     */
-    public IdentityHashSet(int initialCapacity) {
-      this(initialCapacity, DEFAULT_LOAD_FACTOR);
-    }
-    
-    /**
-     * Creates a hash set with the given capacity and load factor.
-     */
-    public IdentityHashSet(int initialCapacity, float loadFactor) {
-      initialCapacity = Math.max(MIN_CAPACITY, initialCapacity);
-      
-      assert initialCapacity > 0 : "Initial capacity must be between (0, "
-          + Integer.MAX_VALUE + "].";
-      assert loadFactor > 0 && loadFactor < 1 : "Load factor must be between (0, 1).";
-      this.loadFactor = loadFactor;
-      allocateBuffers(roundCapacity(initialCapacity));
-    }
-    
-    /**
-     * Adds a reference to the set. Null keys are not allowed.
-     */
-    public boolean add(KType e) {
-      assert e != null : "Null keys not allowed.";
-      
-      if (assigned >= resizeThreshold) expandAndRehash();
-      
-      final int mask = keys.length - 1;
-      int slot = rehash(e) & mask;
-      Object existing;
-      while ((existing = keys[slot]) != null) {
-        if (e == existing) {
-          return false; // already found.
-        }
-        slot = (slot + 1) & mask;
-      }
-      assigned++;
-      keys[slot] = e;
-      return true;
-    }
-
-    /**
-     * Checks if the set contains a given ref.
-     */
-    public boolean contains(KType e) {
-      final int mask = keys.length - 1;
-      int slot = rehash(e) & mask;
-      Object existing;
-      while ((existing = keys[slot]) != null) {
-        if (e == existing) {
-          return true;
-        }
-        slot = (slot + 1) & mask;
-      }
-      return false;
-    }
 
-    /** Rehash via MurmurHash.
-     * 
-     * <p>The implementation is based on the
-     * finalization step from Austin Appleby's
-     * <code>MurmurHash3</code>.
-     * 
-     * @see <a href="http://sites.google.com/site/murmurhash/">http://sites.google.com/site/murmurhash/</a>
-     */
-    private static int rehash(Object o) {
-      int k = System.identityHashCode(o);
-      k ^= k >>> 16;
-      k *= 0x85ebca6b;
-      k ^= k >>> 13;
-      k *= 0xc2b2ae35;
-      k ^= k >>> 16;
-      return k;
-    }
-    
-    /**
-     * Expand the internal storage buffers (capacity) or rehash current keys and
-     * values if there are a lot of deleted slots.
-     */
-    private void expandAndRehash() {
-      final Object[] oldKeys = this.keys;
-      
-      assert assigned >= resizeThreshold;
-      allocateBuffers(nextCapacity(keys.length));
-      
-      /*
-       * Rehash all assigned slots from the old hash table.
-       */
-      final int mask = keys.length - 1;
-      for (int i = 0; i < oldKeys.length; i++) {
-        final Object key = oldKeys[i];
-        if (key != null) {
-          int slot = rehash(key) & mask;
-          while (keys[slot] != null) {
-            slot = (slot + 1) & mask;
-          }
-          keys[slot] = key;
-        }
-      }
-      Arrays.fill(oldKeys, null);
-    }
-    
-    /**
-     * Allocate internal buffers for a given capacity.
-     * 
-     * @param capacity
-     *          New capacity (must be a power of two).
-     */
-    private void allocateBuffers(int capacity) {
-      this.keys = new Object[capacity];
-      this.resizeThreshold = (int) (capacity * DEFAULT_LOAD_FACTOR);
-    }
-    
-    /**
-     * Return the next possible capacity, counting from the current buffers' size.
-     */
-    protected int nextCapacity(int current) {
-      assert current > 0 && Long.bitCount(current) == 1 : "Capacity must be a power of two.";
-      assert ((current << 1) > 0) : "Maximum capacity exceeded ("
-          + (0x80000000 >>> 1) + ").";
-      
-      if (current < MIN_CAPACITY / 2) current = MIN_CAPACITY / 2;
-      return current << 1;
-    }
-    
-    /**
-     * Round the capacity to the next allowed value.
-     */
-    protected int roundCapacity(int requestedCapacity) {
-      // Maximum positive integer that is a power of two.
-      if (requestedCapacity > (0x80000000 >>> 1)) return (0x80000000 >>> 1);
-      
-      int capacity = MIN_CAPACITY;
-      while (capacity < requestedCapacity) {
-        capacity <<= 1;
-      }
-
-      return capacity;
-    }
-    
-    public void clear() {
-      assigned = 0;
-      Arrays.fill(keys, null);
-    }
-    
-    public int size() {
-      return assigned;
-    }
-    
-    public boolean isEmpty() {
-      return size() == 0;
-    }
-
-    @Override
-    public Iterator<KType> iterator() {
-      return new Iterator<KType>() {
-        int pos = -1;
-        Object nextElement = fetchNext();
-
-        @Override
-        public boolean hasNext() {
-          return nextElement != null;
-        }
-
-        @SuppressWarnings("unchecked")
-        @Override
-        public KType next() {
-          Object r = this.nextElement;
-          if (r == null) {
-            throw new NoSuchElementException();
-          }
-          this.nextElement = fetchNext();
-          return (KType) r;
-        }
-
-        private Object fetchNext() {
-          pos++;
-          while (pos < keys.length && keys[pos] == null) {
-            pos++;
-          }
-
-          return (pos >= keys.length ? null : keys[pos]);
-        }
-
-        @Override
-        public void remove() {
-          throw new UnsupportedOperationException();
-        }
-      };
-    }
-  }
 }

Modified: lucene/dev/branches/lucene2878/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/CHANGES.txt?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/CHANGES.txt (original)
+++ lucene/dev/branches/lucene2878/solr/CHANGES.txt Sun Dec 14 22:50:20 2014
@@ -116,6 +116,10 @@ Upgrading from Solr 4.x
   configs.  If you have a strong need to configure this, you must explicitly configure your 
   schema with a custom codec.  See SOLR-6560 and for more details.
 
+* The "checkIntegrityAtMerge" option in solrconfig.xml is now a No-Op and should be removed
+  from any solrconfig.xml files -- these integrity checks are now done automatically at a very
+  low level during the segment merging process.  See SOLR-6834 for more details.
+
 Detailed Change List
 ----------------------
 
@@ -132,8 +136,9 @@ New Features
     modifying solr configuration files.  (Erick Erickson)
   - SOLR-5539: Admin UI - Remove ability to create/modify files (steffkes)
 
-* SOLR-6103: Added DateRangeField for indexing date ranges, especially
-  multi-valued ones. Based on LUCENE-5648. (David Smiley)
+* SOLR-6103: Added DateRangeField for indexing date ranges, especially multi-valued ones.
+  Supports facet.range, DateMath, and is mostly interoperable with TrieDateField.
+  Based on LUCENE-5648. (David Smiley)
 
 * SOLR-6403: TransactionLog replay status logging. (Mark Miller)
 
@@ -228,6 +233,8 @@ New Features
 * SOLR-4799: faster join using join="zipper" aka merge join for nested DIH EntityProcessors
   (Mikhail Khludnev via Noble Paul)
 
+* SOLR-6787: API to manage blobs in Solr (Noble Paul)
+
 Bug Fixes
 ----------------------
 
@@ -310,6 +317,13 @@ Bug Fixes
     implementation instead appends each input piece via the langdetect API.
   (Vitaliy Zhovtyuk, Tomás Fernández Löbbe, Rob Tulloh, Steve Rowe)
 
+* SOLR-6626: NPE in FieldMutatingUpdateProcessor when indexing a doc with
+  null field value (Noble Paul)
+
+* SOLR-6604: SOLR-6812: Fix NPE with distrib.singlePass=true and expand
+  component. Increased test coverage of expand component with docValues.
+  (Christine Poerschke, Per Steffensen, shalin)
+
 Optimizations
 ----------------------
 
@@ -464,6 +478,20 @@ Other Changes
 * SOLR-6560: Purge termIndexInterval from example/test configs
   (Tom Burton-West, hossman)
 
+* SOLR-6773: Remove the multicore example as the DIH and cloud examples 
+  illustrate multicore behavior (hossman, Timothy Potter)
+
+* SOLR-6834: Warn if checkIntegrityAtMerge is configured.  This option is no longer meaningful
+  since the checks are done automatically at a very low level in the segment merging. 
+  This warning will become an error in Solr 6.0.  (hossman)
+
+* SOLR-6833: Examples started with bin/solr -e should use a solr.solr.home directory under
+  the example directory instead of server/solr. (Alexandre Rafalovitch, Anshum Gupta, hossman,
+  Timothy Potter)
+
+* SOLR-6826: fieldType capitalization is not consistent with the rest of case-sensitive field names.
+  (Alexandre Rafalovitch via Erick Erickson)
+
 ==================  4.10.3 ==================
 
 Bug Fixes
@@ -545,6 +573,8 @@ Other Changes
 * SOLR-6653: bin/solr script should return error code >0 when something fails
   (janhoy, Timothy Potter)
 
+* SOLR-6829: Added getter/setter for lastException in DIH's ContextImpl (ehatcher)
+
 ==================  4.10.2 ==================
 
 Bug FixesAnalyticsComponent

Modified: lucene/dev/branches/lucene2878/solr/README.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/README.txt?rev=1645537&r1=1645536&r2=1645537&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/README.txt (original)
+++ lucene/dev/branches/lucene2878/solr/README.txt Sun Dec 14 22:50:20 2014
@@ -30,14 +30,29 @@ Getting Started
 
 To start Solr for the first time after installation, simply do:
 
-  bin/solr start -f
+  bin/solr start
 
-This will launch a Solr server in the foreground of your shell, bound
-to port 8983. Alternatively, you can launch Solr in "cloud" mode,
-which allows you to scale out using sharding and replication. To
-launch Solr in cloud mode, do:
+This will launch a Solr server in the background of your shell, bound
+to port 8983. After starting Solr, you can create a new core for indexing
+your data by doing:
 
-  bin/solr start -f -cloud
+  bin/solr create_core -n <name>
+
+This will create a core that uses a data-driven schema which tries to guess
+the correct field type when you add documents to the index. To see all available
+options for creating a new core, execute:
+
+  bin/solr create_core -help
+
+Alternatively, you can launch Solr in "cloud" mode, which allows you to scale
+out using sharding and replication. To launch Solr in cloud mode, do:
+
+  bin/solr start -cloud
+
+After starting Solr in cloud mode, you can create a new collection for indexing
+your data by doing:
+
+  bin/solr create_collection -n <name>
 
 To see all available options for starting Solr, please do:
 
@@ -47,6 +62,27 @@ After starting Solr, direct your Web bro
 
   http://localhost:8983/solr/
 
+
+Solr Examples
+---------------
+
+Solr includes a few examples to help you get started. To run a specific example, do:
+
+  bin/solr -e <EXAMPLE> where <EXAMPLE> is one of:
+
+    cloud        : SolrCloud example
+    dih          : Data Import Handler (rdbms, mail, rss, tika)
+    schemaless   : Schema-less example (schema is inferred from data during indexing)
+    techproducts : Kitchen sink example providing comprehensive examples of Solr features
+
+For instance, if you want to run the Solr Data Import Handler example, do:
+
+  bin/solr -e dih
+
+
+Indexing Documents
+---------------
+
 To add documents to the index, use the post.jar (or post.sh script) in
 the example/exampledocs subdirectory (while Solr is running), for example: