You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/11/19 03:09:26 UTC

svn commit: r1640464 - in /lucene/dev/trunk: lucene/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/test/org/apache/lucene/ lucene/core/src/test/org/apache/lucene/index/ lucene/core/src/test/org/apache/lucene/search/ lucene/expressions/s...

Author: rmuir
Date: Wed Nov 19 02:09:25 2014
New Revision: 1640464

URL: http://svn.apache.org/r1640464
Log:
LUCENE-6062: throw exception instead of doing nothing, when sorting/grouping etc on misconfigured field

Added:
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/TestSearch.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java
    lucene/dev/trunk/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSorts.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
    lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
    lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupingSearchTest.java
    lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
    lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
    lucene/dev/trunk/lucene/queries/src/test/org/apache/lucene/queries/function/TestBoostedQuery.java
    lucene/dev/trunk/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Wed Nov 19 02:09:25 2014
@@ -206,6 +206,10 @@ API Changes
   instance based on a Directory implementation passed to the factory method.
   See MIGRATE.txt for more details.  (Uwe Schindler, Robert Muir)
 
+* LUCENE-6062: Throw exception instead of silently doing nothing if you try to 
+  sort/group/etc on a misconfigured field (e.g. no docvalues, no UninvertingReader, etc).  
+  (Robert Muir)
+
 Bug Fixes
 
 * LUCENE-5650: Enforce read-only access to any path outside the temporary

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java Wed Nov 19 02:09:25 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -199,12 +200,31 @@ public final class DocValues {
   // some helpers, for transition from fieldcache apis.
   // as opposed to the LeafReader apis (which must be strict for consistency), these are lenient
   
+  // helper method: to give a nice error when LeafReader.getXXXDocValues returns null.
+  private static void checkField(LeafReader in, String field, DocValuesType... expected) {
+    FieldInfo fi = in.getFieldInfos().fieldInfo(field);
+    if (fi != null) {
+      DocValuesType actual = fi.getDocValuesType();
+      throw new IllegalStateException("unexpected docvalues type " + actual + 
+                                        " for field '" + field + "' " +
+                                        (expected.length == 1 
+                                        ? "(expected=" + expected[0]
+                                        : "(expected one of " + Arrays.toString(expected)) + "). " +
+                                        "Use UninvertingReader or index with docvalues.");
+    }
+  }
+  
   /**
-   * Returns NumericDocValues for the reader, or {@link #emptyNumeric()} if it has none. 
+   * Returns NumericDocValues for the field, or {@link #emptyNumeric()} if it has none. 
+   * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
+   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
+   * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#NUMERIC}.
+   * @throws IOException if an I/O error occurs.
    */
-  public static NumericDocValues getNumeric(LeafReader in, String field) throws IOException {
-    NumericDocValues dv = in.getNumericDocValues(field);
+  public static NumericDocValues getNumeric(LeafReader reader, String field) throws IOException {
+    NumericDocValues dv = reader.getNumericDocValues(field);
     if (dv == null) {
+      checkField(reader, field, DocValuesType.NUMERIC);
       return emptyNumeric();
     } else {
       return dv;
@@ -212,13 +232,19 @@ public final class DocValues {
   }
   
   /**
-   * Returns BinaryDocValues for the reader, or {@link #emptyBinary} if it has none. 
+   * Returns BinaryDocValues for the field, or {@link #emptyBinary} if it has none. 
+   * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
+   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
+   * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#BINARY}
+   *                               or {@link DocValuesType#SORTED}.
+   * @throws IOException if an I/O error occurs.
    */
-  public static BinaryDocValues getBinary(LeafReader in, String field) throws IOException {
-    BinaryDocValues dv = in.getBinaryDocValues(field);
+  public static BinaryDocValues getBinary(LeafReader reader, String field) throws IOException {
+    BinaryDocValues dv = reader.getBinaryDocValues(field);
     if (dv == null) {
-      dv = in.getSortedDocValues(field);
+      dv = reader.getSortedDocValues(field);
       if (dv == null) {
+        checkField(reader, field, DocValuesType.BINARY, DocValuesType.SORTED);
         return emptyBinary();
       }
     }
@@ -226,11 +252,16 @@ public final class DocValues {
   }
   
   /**
-   * Returns SortedDocValues for the reader, or {@link #emptySorted} if it has none. 
+   * Returns SortedDocValues for the field, or {@link #emptySorted} if it has none. 
+   * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
+   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
+   * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#SORTED}.
+   * @throws IOException if an I/O error occurs.
    */
-  public static SortedDocValues getSorted(LeafReader in, String field) throws IOException {
-    SortedDocValues dv = in.getSortedDocValues(field);
+  public static SortedDocValues getSorted(LeafReader reader, String field) throws IOException {
+    SortedDocValues dv = reader.getSortedDocValues(field);
     if (dv == null) {
+      checkField(reader, field, DocValuesType.SORTED);
       return emptySorted();
     } else {
       return dv;
@@ -238,29 +269,41 @@ public final class DocValues {
   }
   
   /**
-   * Returns SortedNumericDocValues for the reader, or {@link #emptySortedNumeric} if it has none. 
+   * Returns SortedNumericDocValues for the field, or {@link #emptySortedNumeric} if it has none. 
+   * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
+   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
+   * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#SORTED_NUMERIC}
+   *                               or {@link DocValuesType#NUMERIC}.
+   * @throws IOException if an I/O error occurs.
    */
-  public static SortedNumericDocValues getSortedNumeric(LeafReader in, String field) throws IOException {
-    SortedNumericDocValues dv = in.getSortedNumericDocValues(field);
+  public static SortedNumericDocValues getSortedNumeric(LeafReader reader, String field) throws IOException {
+    SortedNumericDocValues dv = reader.getSortedNumericDocValues(field);
     if (dv == null) {
-      NumericDocValues single = in.getNumericDocValues(field);
+      NumericDocValues single = reader.getNumericDocValues(field);
       if (single == null) {
-        return emptySortedNumeric(in.maxDoc());
+        checkField(reader, field, DocValuesType.SORTED_NUMERIC, DocValuesType.NUMERIC);
+        return emptySortedNumeric(reader.maxDoc());
       }
-      Bits bits = in.getDocsWithField(field);
+      Bits bits = reader.getDocsWithField(field);
       return singleton(single, bits);
     }
     return dv;
   }
   
   /**
-   * Returns SortedSetDocValues for the reader, or {@link #emptySortedSet} if it has none. 
+   * Returns SortedSetDocValues for the field, or {@link #emptySortedSet} if it has none. 
+   * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
+   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
+   * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#SORTED_SET}
+   *                               or {@link DocValuesType#SORTED}.
+   * @throws IOException if an I/O error occurs.
    */
-  public static SortedSetDocValues getSortedSet(LeafReader in, String field) throws IOException {
-    SortedSetDocValues dv = in.getSortedSetDocValues(field);
+  public static SortedSetDocValues getSortedSet(LeafReader reader, String field) throws IOException {
+    SortedSetDocValues dv = reader.getSortedSetDocValues(field);
     if (dv == null) {
-      SortedDocValues sorted = in.getSortedDocValues(field);
+      SortedDocValues sorted = reader.getSortedDocValues(field);
       if (sorted == null) {
+        checkField(reader, field, DocValuesType.SORTED, DocValuesType.SORTED_SET);
         return emptySortedSet();
       }
       return singleton(sorted);
@@ -269,12 +312,21 @@ public final class DocValues {
   }
   
   /**
-   * Returns Bits for the reader, or {@link Bits} matching nothing if it has none. 
+   * Returns Bits for the field, or {@link Bits} matching nothing if it has none. 
+   * @return bits instance, or an empty instance if {@code field} does not exist in this reader.
+   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
+   * @throws IOException if an I/O error occurs.
    */
-  public static Bits getDocsWithField(LeafReader in, String field) throws IOException {
-    Bits dv = in.getDocsWithField(field);
+  public static Bits getDocsWithField(LeafReader reader, String field) throws IOException {
+    Bits dv = reader.getDocsWithField(field);
     if (dv == null) {
-      return new Bits.MatchNoBits(in.maxDoc());
+      assert DocValuesType.values().length == 6; // we just don't want NONE
+      checkField(reader, field, DocValuesType.BINARY, 
+                            DocValuesType.NUMERIC, 
+                            DocValuesType.SORTED, 
+                            DocValuesType.SORTED_NUMERIC, 
+                            DocValuesType.SORTED_SET);
+      return new Bits.MatchNoBits(reader.maxDoc());
     } else {
       return dv;
     }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/TestSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/TestSearch.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/TestSearch.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/TestSearch.java Wed Nov 19 02:09:25 2014
@@ -128,6 +128,7 @@ public class TestSearch extends LuceneTe
         Document d = new Document();
         d.add(newTextField("contents", docs[j], Field.Store.YES));
         d.add(new IntField("id", j, Field.Store.NO));
+        d.add(new NumericDocValuesField("id", j));
         writer.addDocument(d);
       }
       writer.close();

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java?rev=1640464&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java Wed Nov 19 02:09:25 2014
@@ -0,0 +1,287 @@
+package org.apache.lucene.index;
+
+/*
+ * 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.
+ */
+
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+/** Tests helper methods in DocValues */
+public class TestDocValues extends LuceneTestCase {
+  
+  /** 
+   * If the field doesn't exist, we return empty instances:
+   * it can easily happen that a segment just doesn't have any docs with the field.
+   */
+  public void testEmptyIndex() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
+    iw.addDocument(new Document());
+    DirectoryReader dr = DirectoryReader.open(iw, true);
+    LeafReader r = getOnlySegmentReader(dr);
+    
+    // ok
+    assertNotNull(DocValues.getBinary(r, "bogus"));
+    assertNotNull(DocValues.getNumeric(r, "bogus"));
+    assertNotNull(DocValues.getSorted(r, "bogus"));
+    assertNotNull(DocValues.getSortedSet(r, "bogus"));
+    assertNotNull(DocValues.getSortedNumeric(r, "bogus"));
+    assertNotNull(DocValues.getDocsWithField(r, "bogus"));
+    
+    dr.close();
+    iw.close();
+    dir.close();
+  }
+  
+  /** 
+   * field just doesnt have any docvalues at all: exception
+   */
+  public void testMisconfiguredField() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
+    Document doc = new Document();
+    doc.add(new StringField("foo", "bar", Field.Store.NO));
+    iw.addDocument(doc);
+    DirectoryReader dr = DirectoryReader.open(iw, true);
+    LeafReader r = getOnlySegmentReader(dr);
+   
+    // errors
+    try {
+      DocValues.getBinary(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getNumeric(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSorted(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSortedSet(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSortedNumeric(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getDocsWithField(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    
+    dr.close();
+    iw.close();
+    dir.close();
+  }
+  
+  /** 
+   * field with numeric docvalues
+   */
+  public void testNumericField() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("foo", 3));
+    iw.addDocument(doc);
+    DirectoryReader dr = DirectoryReader.open(iw, true);
+    LeafReader r = getOnlySegmentReader(dr);
+    
+    // ok
+    assertNotNull(DocValues.getNumeric(r, "foo"));
+    assertNotNull(DocValues.getSortedNumeric(r, "foo"));
+    assertNotNull(DocValues.getDocsWithField(r, "foo"));
+    
+    // errors
+    try {
+      DocValues.getBinary(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSorted(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSortedSet(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    
+    dr.close();
+    iw.close();
+    dir.close();
+  }
+  
+  /** 
+   * field with binary docvalues
+   */
+  public void testBinaryField() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
+    Document doc = new Document();
+    doc.add(new BinaryDocValuesField("foo", new BytesRef("bar")));
+    iw.addDocument(doc);
+    DirectoryReader dr = DirectoryReader.open(iw, true);
+    LeafReader r = getOnlySegmentReader(dr);
+    
+    // ok
+    assertNotNull(DocValues.getBinary(r, "foo"));
+    assertNotNull(DocValues.getDocsWithField(r, "foo"));
+    
+    // errors
+    try {
+      DocValues.getNumeric(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSorted(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSortedSet(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSortedNumeric(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    
+    dr.close();
+    iw.close();
+    dir.close();
+  }
+  
+  /** 
+   * field with sorted docvalues
+   */
+  public void testSortedField() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
+    Document doc = new Document();
+    doc.add(new SortedDocValuesField("foo", new BytesRef("bar")));
+    iw.addDocument(doc);
+    DirectoryReader dr = DirectoryReader.open(iw, true);
+    LeafReader r = getOnlySegmentReader(dr);
+    
+    // ok
+    assertNotNull(DocValues.getBinary(r, "foo"));
+    assertNotNull(DocValues.getSorted(r, "foo"));
+    assertNotNull(DocValues.getSortedSet(r, "foo"));
+    assertNotNull(DocValues.getDocsWithField(r, "foo"));
+    
+    // errors
+    try {
+      DocValues.getNumeric(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSortedNumeric(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    
+    dr.close();
+    iw.close();
+    dir.close();
+  }
+  
+  /** 
+   * field with sortedset docvalues
+   */
+  public void testSortedSetField() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
+    Document doc = new Document();
+    doc.add(new SortedSetDocValuesField("foo", new BytesRef("bar")));
+    iw.addDocument(doc);
+    DirectoryReader dr = DirectoryReader.open(iw, true);
+    LeafReader r = getOnlySegmentReader(dr);
+    
+    // ok
+    assertNotNull(DocValues.getSortedSet(r, "foo"));
+    assertNotNull(DocValues.getDocsWithField(r, "foo"));
+    
+    // errors
+    try {
+      DocValues.getBinary(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getNumeric(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSorted(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSortedNumeric(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    
+    dr.close();
+    iw.close();
+    dir.close();
+  }
+  
+  /** 
+   * field with sortednumeric docvalues
+   */
+  public void testSortedNumericField() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
+    Document doc = new Document();
+    doc.add(new SortedNumericDocValuesField("foo", 3));
+    iw.addDocument(doc);
+    DirectoryReader dr = DirectoryReader.open(iw, true);
+    LeafReader r = getOnlySegmentReader(dr);
+    
+    // ok
+    assertNotNull(DocValues.getSortedNumeric(r, "foo"));
+    assertNotNull(DocValues.getDocsWithField(r, "foo"));
+    
+    // errors
+    try {
+      DocValues.getBinary(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getNumeric(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSorted(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    try {
+      DocValues.getSortedSet(r, "foo");
+      fail();
+    } catch (IllegalStateException expected) {}
+    
+    dr.close();
+    iw.close();
+    dir.close();
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java Wed Nov 19 02:09:25 2014
@@ -28,10 +28,12 @@ import java.util.TreeMap;
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 
 /** Unit test for sorting code. */
@@ -58,7 +60,7 @@ public class TestCustomSearcherSort exte
       Document doc = new Document();
       if ((i % 5) != 0) { // some documents must not have an entry in the first
                           // sort field
-        doc.add(newStringField("publicationDate_", random.getLuceneDate(), Field.Store.YES));
+        doc.add(new SortedDocValuesField("publicationDate_", new BytesRef(random.getLuceneDate())));
       }
       if ((i % 7) == 0) { // some documents to match the query (see below)
         doc.add(newTextField("content", "test", Field.Store.YES));

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java Wed Nov 19 02:09:25 2014
@@ -24,6 +24,7 @@ import java.util.Random;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DocsEnum;
@@ -35,6 +36,7 @@ import org.apache.lucene.search.Filtered
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
@@ -65,21 +67,25 @@ public class TestFilteredQuery extends L
     Document doc = new Document();
     doc.add (newTextField("field", "one two three four five", Field.Store.YES));
     doc.add (newTextField("sorter", "b", Field.Store.YES));
+    doc.add (new SortedDocValuesField("sorter", new BytesRef("b")));
     writer.addDocument (doc);
 
     doc = new Document();
     doc.add (newTextField("field", "one two three four", Field.Store.YES));
     doc.add (newTextField("sorter", "d", Field.Store.YES));
+    doc.add (new SortedDocValuesField("sorter", new BytesRef("d")));
     writer.addDocument (doc);
 
     doc = new Document();
     doc.add (newTextField("field", "one two three y", Field.Store.YES));
     doc.add (newTextField("sorter", "a", Field.Store.YES));
+    doc.add (new SortedDocValuesField("sorter", new BytesRef("a")));
     writer.addDocument (doc);
 
     doc = new Document();
     doc.add (newTextField("field", "one two x", Field.Store.YES));
     doc.add (newTextField("sorter", "c", Field.Store.YES));
+    doc.add (new SortedDocValuesField("sorter", new BytesRef("c")));
     writer.addDocument (doc);
 
     // tests here require single segment (eg try seed

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java Wed Nov 19 02:09:25 2014
@@ -24,6 +24,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
@@ -48,6 +49,7 @@ public class TestIndexSearcher extends L
       Document doc = new Document();
       doc.add(newStringField("field", Integer.toString(i), Field.Store.NO));
       doc.add(newStringField("field2", Boolean.toString(i % 2 == 0), Field.Store.NO));
+      doc.add(new SortedDocValuesField("field2", new BytesRef(Boolean.toString(i % 2 == 0))));
       iw.addDocument(doc);
     }
     reader = iw.getReader();

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java Wed Nov 19 02:09:25 2014
@@ -21,15 +21,13 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Random;
 
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.DoubleField;
+import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FloatDocValuesField;
-import org.apache.lucene.document.FloatField;
-import org.apache.lucene.document.IntField;
-import org.apache.lucene.document.LongField;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StoredField;
@@ -63,22 +61,12 @@ public class TestSearchAfter extends Luc
           new SortField("double", SortField.Type.DOUBLE, false),
           new SortField("bytes", SortField.Type.STRING, false),
           new SortField("bytesval", SortField.Type.STRING_VAL, false),
-          new SortField("intdocvalues", SortField.Type.INT, false),
-          new SortField("floatdocvalues", SortField.Type.FLOAT, false),
-          new SortField("sortedbytesdocvalues", SortField.Type.STRING, false),
-          new SortField("sortedbytesdocvaluesval", SortField.Type.STRING_VAL, false),
-          new SortField("straightbytesdocvalues", SortField.Type.STRING_VAL, false),
           new SortField("int", SortField.Type.INT, true),
           new SortField("long", SortField.Type.LONG, true),
           new SortField("float", SortField.Type.FLOAT, true),
           new SortField("double", SortField.Type.DOUBLE, true),
           new SortField("bytes", SortField.Type.STRING, true),
           new SortField("bytesval", SortField.Type.STRING_VAL, true),
-          new SortField("intdocvalues", SortField.Type.INT, true),
-          new SortField("floatdocvalues", SortField.Type.FLOAT, true),
-          new SortField("sortedbytesdocvalues", SortField.Type.STRING, true),
-          new SortField("sortedbytesdocvaluesval", SortField.Type.STRING_VAL, true),
-          new SortField("straightbytesdocvalues", SortField.Type.STRING_VAL, true),
           SortField.FIELD_SCORE,
           SortField.FIELD_DOC,
         }));
@@ -136,26 +124,19 @@ public class TestSearchAfter extends Luc
     dir = newDirectory();
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
     int numDocs = atLeast(200);
+    Random r = random();
     for (int i = 0; i < numDocs; i++) {
       List<Field> fields = new ArrayList<>();
       fields.add(newTextField("english", English.intToEnglish(i), Field.Store.NO));
       fields.add(newTextField("oddeven", (i % 2 == 0) ? "even" : "odd", Field.Store.NO));
-      fields.add(newStringField("byte", "" + ((byte) random().nextInt()), Field.Store.NO));
-      fields.add(newStringField("short", "" + ((short) random().nextInt()), Field.Store.NO));
-      fields.add(new IntField("int", random().nextInt(), Field.Store.NO));
-      fields.add(new LongField("long", random().nextLong(), Field.Store.NO));
-
-      fields.add(new FloatField("float", random().nextFloat(), Field.Store.NO));
-      fields.add(new DoubleField("double", random().nextDouble(), Field.Store.NO));
-      fields.add(newStringField("bytes", TestUtil.randomRealisticUnicodeString(random()), Field.Store.NO));
-      fields.add(newStringField("bytesval", TestUtil.randomRealisticUnicodeString(random()), Field.Store.NO));
-      fields.add(new DoubleField("double", random().nextDouble(), Field.Store.NO));
-
-      fields.add(new NumericDocValuesField("intdocvalues", random().nextInt()));
-      fields.add(new FloatDocValuesField("floatdocvalues", random().nextFloat()));
-      fields.add(new SortedDocValuesField("sortedbytesdocvalues", new BytesRef(TestUtil.randomRealisticUnicodeString(random()))));
-      fields.add(new SortedDocValuesField("sortedbytesdocvaluesval", new BytesRef(TestUtil.randomRealisticUnicodeString(random()))));
-      fields.add(new BinaryDocValuesField("straightbytesdocvalues", new BytesRef(TestUtil.randomRealisticUnicodeString(random()))));
+      fields.add(new NumericDocValuesField("byte", (byte) r.nextInt()));
+      fields.add(new NumericDocValuesField("short", (short) r.nextInt()));
+      fields.add(new NumericDocValuesField("int", r.nextInt()));
+      fields.add(new NumericDocValuesField("long", r.nextLong()));
+      fields.add(new FloatDocValuesField("float", r.nextFloat()));
+      fields.add(new DoubleDocValuesField("double", r.nextDouble()));
+      fields.add(new SortedDocValuesField("bytes", new BytesRef(TestUtil.randomRealisticUnicodeString(random()))));
+      fields.add(new BinaryDocValuesField("bytesval", new BytesRef(TestUtil.randomRealisticUnicodeString(random()))));
 
       Document document = new Document();
       document.add(new StoredField("id", ""+i));

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java Wed Nov 19 02:09:25 2014
@@ -219,9 +219,9 @@ public class TestShardSearching extends 
                 //sort = new Sort(SortField.FIELD_DOC);
                 sort = null;
               } else if (what == 2) {
-                sort = new Sort(new SortField[] {new SortField("docid_int", SortField.Type.INT, random().nextBoolean())});
+                sort = new Sort(new SortField[] {new SortField("docid_intDV", SortField.Type.INT, random().nextBoolean())});
               } else {
-                sort = new Sort(new SortField[] {new SortField("title", SortField.Type.STRING, random().nextBoolean())});
+                sort = new Sort(new SortField[] {new SortField("titleDV", SortField.Type.STRING, random().nextBoolean())});
               }
             }
           } else {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java Wed Nov 19 02:09:25 2014
@@ -19,8 +19,11 @@ package org.apache.lucene.search;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FloatDocValuesField;
 import org.apache.lucene.document.FloatField;
 import org.apache.lucene.document.IntField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.CompositeReaderContext;
 import org.apache.lucene.index.IndexReader;
@@ -29,6 +32,7 @@ import org.apache.lucene.index.RandomInd
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -103,9 +107,9 @@ public class TestTopDocsMerge extends Lu
 
       for(int docIDX=0;docIDX<numDocs;docIDX++) {
         final Document doc = new Document();
-        doc.add(newStringField("string", TestUtil.randomRealisticUnicodeString(random()), Field.Store.NO));
+        doc.add(new SortedDocValuesField("string", new BytesRef(TestUtil.randomRealisticUnicodeString(random()))));
         doc.add(newTextField("text", content[random().nextInt(content.length)], Field.Store.NO));
-        doc.add(new FloatField("float", random().nextFloat(), Field.Store.NO));
+        doc.add(new FloatDocValuesField("float", random().nextFloat()));
         final int intValue;
         if (random().nextInt(100) == 17) {
           intValue = Integer.MIN_VALUE;
@@ -114,7 +118,7 @@ public class TestTopDocsMerge extends Lu
         } else {
           intValue = random().nextInt();
         }
-        doc.add(new IntField("int", intValue, Field.Store.NO));
+        doc.add(new NumericDocValuesField("int", intValue));
         if (VERBOSE) {
           System.out.println("  doc=" + doc);
         }

Modified: lucene/dev/trunk/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSorts.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSorts.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSorts.java (original)
+++ lucene/dev/trunk/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSorts.java Wed Nov 19 02:09:25 2014
@@ -21,12 +21,9 @@ import java.util.Arrays;
 import java.util.Collections;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.DoubleField;
+import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FloatDocValuesField;
-import org.apache.lucene.document.FloatField;
-import org.apache.lucene.document.IntField;
-import org.apache.lucene.document.LongField;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.expressions.js.JavascriptCompiler;
 import org.apache.lucene.index.IndexReader;
@@ -68,16 +65,10 @@ public class TestExpressionSorts extends
       Document document = new Document();
       document.add(newTextField("english", English.intToEnglish(i), Field.Store.NO));
       document.add(newTextField("oddeven", (i % 2 == 0) ? "even" : "odd", Field.Store.NO));
-      document.add(newStringField("byte", "" + ((byte) random().nextInt()), Field.Store.NO));
-      document.add(newStringField("short", "" + ((short) random().nextInt()), Field.Store.NO));
-      document.add(new IntField("int", random().nextInt(), Field.Store.NO));
-      document.add(new LongField("long", random().nextLong(), Field.Store.NO));
-
-      document.add(new FloatField("float", random().nextFloat(), Field.Store.NO));
-      document.add(new DoubleField("double", random().nextDouble(), Field.Store.NO));
-
-      document.add(new NumericDocValuesField("intdocvalues", random().nextInt()));
-      document.add(new FloatDocValuesField("floatdocvalues", random().nextFloat()));
+      document.add(new NumericDocValuesField("int", random().nextInt()));
+      document.add(new NumericDocValuesField("long", random().nextLong()));
+      document.add(new FloatDocValuesField("float", random().nextFloat()));
+      document.add(new DoubleDocValuesField("double", random().nextDouble()));
       iw.addDocument(document);
     }
     reader = iw.getReader();
@@ -119,8 +110,6 @@ public class TestExpressionSorts extends
           new SortField("long", SortField.Type.LONG, reversed),
           new SortField("float", SortField.Type.FLOAT, reversed),
           new SortField("double", SortField.Type.DOUBLE, reversed),
-          new SortField("intdocvalues", SortField.Type.INT, reversed),
-          new SortField("floatdocvalues", SortField.Type.FLOAT, reversed),
           new SortField("score", SortField.Type.SCORE)
       };
       Collections.shuffle(Arrays.asList(fields), random());

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java Wed Nov 19 02:09:25 2014
@@ -27,7 +27,6 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FloatDocValuesField;
-import org.apache.lucene.document.IntField;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.facet.FacetField;
@@ -209,7 +208,7 @@ public class TestTaxonomyFacetSumValueSo
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
 
     Document doc = new Document();
-    doc.add(new IntField("num", 10, Field.Store.NO));
+    doc.add(new NumericDocValuesField("num", 10));
     doc.add(new FacetField("a", "foo1"));
     writer.addDocument(config.build(taxoWriter, doc));
 

Modified: lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java (original)
+++ lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java Wed Nov 19 02:09:25 2014
@@ -440,6 +440,7 @@ public class DistinctValuesCollectorTest
 
       Document doc = new Document();
       doc.add(new StringField("id", String.format(Locale.ROOT, "%09d", i), Field.Store.YES));
+      doc.add(new SortedDocValuesField("id", new BytesRef(String.format(Locale.ROOT, "%09d", i))));
       if (groupValue != null) {
         addField(doc, groupField, groupValue);
       }

Modified: lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupingSearchTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupingSearchTest.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupingSearchTest.java (original)
+++ lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupingSearchTest.java Wed Nov 19 02:09:25 2014
@@ -229,6 +229,7 @@ public class GroupingSearchTest extends 
         newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
     Document doc = new Document();
     doc.add(newField("group", "foo", StringField.TYPE_NOT_STORED));
+    doc.add(new SortedDocValuesField("group", new BytesRef("foo")));
     w.addDocument(doc);
 
     IndexSearcher indexSearcher = newSearcher(w.getReader());

Modified: lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java (original)
+++ lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java Wed Nov 19 02:09:25 2014
@@ -1284,6 +1284,7 @@ public class TestBlockJoin extends Lucen
     RandomIndexWriter w = new RandomIndexWriter(random(), d);
     Document parent = new Document();
     parent.add(new StoredField("parentID", "0"));
+    parent.add(new SortedDocValuesField("parentID", new BytesRef("0")));
     parent.add(newTextField("parentText", "text", Field.Store.NO));
     parent.add(newStringField("isParent", "yes", Field.Store.NO));
 
@@ -1304,6 +1305,7 @@ public class TestBlockJoin extends Lucen
     parent.add(newTextField("parentText", "text", Field.Store.NO));
     parent.add(newStringField("isParent", "yes", Field.Store.NO));
     parent.add(new StoredField("parentID", "1"));
+    parent.add(new SortedDocValuesField("parentID", new BytesRef("1")));
 
     // parent last:
     docs.add(parent);
@@ -1348,6 +1350,7 @@ public class TestBlockJoin extends Lucen
     RandomIndexWriter w = new RandomIndexWriter(random(), d);
     Document parent = new Document();
     parent.add(new StoredField("parentID", "0"));
+    parent.add(new SortedDocValuesField("parentID", new BytesRef("0")));
     parent.add(newTextField("parentText", "text", Field.Store.NO));
     parent.add(newStringField("isParent", "yes", Field.Store.NO));
 
@@ -1368,6 +1371,8 @@ public class TestBlockJoin extends Lucen
     parent.add(newTextField("parentText", "text", Field.Store.NO));
     parent.add(newStringField("isParent", "yes", Field.Store.NO));
     parent.add(new StoredField("parentID", "1"));
+    parent.add(new SortedDocValuesField("parentID", new BytesRef("1")));
+    
 
     // parent last:
     docs.add(parent);

Modified: lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (original)
+++ lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java Wed Nov 19 02:09:25 2014
@@ -238,12 +238,14 @@ public class TestJoinUtil extends Lucene
     doc.add(new TextField("description", "random text", Field.Store.NO));
     doc.add(new TextField("name", "name1", Field.Store.NO));
     doc.add(new TextField(idField, "7", Field.Store.NO));
+    doc.add(new SortedDocValuesField(idField, new BytesRef("7")));
     w.addDocument(doc);
 
     // 1
     doc = new Document();
     doc.add(new TextField("price", "10.0", Field.Store.NO));
     doc.add(new TextField(idField, "2", Field.Store.NO));
+    doc.add(new SortedDocValuesField(idField, new BytesRef("2")));
     doc.add(new TextField(toField, "7", Field.Store.NO));
     w.addDocument(doc);
 
@@ -251,6 +253,7 @@ public class TestJoinUtil extends Lucene
     doc = new Document();
     doc.add(new TextField("price", "20.0", Field.Store.NO));
     doc.add(new TextField(idField, "3", Field.Store.NO));
+    doc.add(new SortedDocValuesField(idField, new BytesRef("3")));
     doc.add(new TextField(toField, "7", Field.Store.NO));
     w.addDocument(doc);
 
@@ -266,6 +269,7 @@ public class TestJoinUtil extends Lucene
     doc = new Document();
     doc.add(new TextField("price", "10.0", Field.Store.NO));
     doc.add(new TextField(idField, "5", Field.Store.NO));
+    doc.add(new SortedDocValuesField(idField, new BytesRef("5")));
     doc.add(new TextField(toField, "0", Field.Store.NO));
     w.addDocument(doc);
 
@@ -273,6 +277,7 @@ public class TestJoinUtil extends Lucene
     doc = new Document();
     doc.add(new TextField("price", "20.0", Field.Store.NO));
     doc.add(new TextField(idField, "6", Field.Store.NO));
+    doc.add(new SortedDocValuesField(idField, new BytesRef("6")));
     doc.add(new TextField(toField, "0", Field.Store.NO));
     w.addDocument(doc);
 

Modified: lucene/dev/trunk/lucene/queries/src/test/org/apache/lucene/queries/function/TestBoostedQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queries/src/test/org/apache/lucene/queries/function/TestBoostedQuery.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queries/src/test/org/apache/lucene/queries/function/TestBoostedQuery.java (original)
+++ lucene/dev/trunk/lucene/queries/src/test/org/apache/lucene/queries/function/TestBoostedQuery.java Wed Nov 19 02:09:25 2014
@@ -4,7 +4,7 @@ package org.apache.lucene.queries.functi
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
@@ -18,6 +18,7 @@ import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -55,7 +56,7 @@ public class TestBoostedQuery extends Lu
     iwConfig.setMergePolicy(newLogMergePolicy());
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConfig);
     Document document = new Document();
-    Field idField = new StringField("id", "", Field.Store.NO);
+    Field idField = new SortedDocValuesField("id", new BytesRef());
     document.add(idField);
     iw.addDocument(document);
     ir = iw.getReader();

Modified: lucene/dev/trunk/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java (original)
+++ lucene/dev/trunk/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java Wed Nov 19 02:09:25 2014
@@ -24,6 +24,7 @@ import com.spatial4j.core.shape.Shape;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.IntField;
+import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
@@ -122,7 +123,8 @@ public class SpatialExample extends Luce
 
   private Document newSampleDocument(int id, Shape... shapes) {
     Document doc = new Document();
-    doc.add(new IntField("id", id, Field.Store.YES));
+    doc.add(new StoredField("id", id));
+    doc.add(new NumericDocValuesField("id", id));
     //Potentially more than one shape in this field is supported by some
     // strategies; see the javadocs of the SpatialStrategy impl to see.
     for (Shape shape : shapes) {

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java Wed Nov 19 02:09:25 2014
@@ -663,8 +663,8 @@ public abstract class ThreadedIndexingAn
 
   private int runQuery(IndexSearcher s, Query q) throws Exception {
     s.search(q, 10);
-    int hitCount = s.search(q, null, 10, new Sort(new SortField("title", SortField.Type.STRING))).totalHits;
-    final Sort dvSort = new Sort(new SortField("title", SortField.Type.STRING));
+    int hitCount = s.search(q, null, 10, new Sort(new SortField("titleDV", SortField.Type.STRING))).totalHits;
+    final Sort dvSort = new Sort(new SortField("titleDV", SortField.Type.STRING));
     int hitCount2 = s.search(q, null, 10, dvSort).totalHits;
     assertEquals(hitCount, hitCount2);
     return hitCount;

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java Wed Nov 19 02:09:25 2014
@@ -38,6 +38,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.IntField;
+import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
@@ -162,6 +163,7 @@ public class LineFileDocs implements Clo
     final Field body;
     final Field id;
     final Field idNum;
+    final Field idNumDV;
     final Field date;
 
     public DocState(boolean useDocValues) {
@@ -193,9 +195,12 @@ public class LineFileDocs implements Clo
 
       if (useDocValues) {
         titleDV = new SortedDocValuesField("titleDV", new BytesRef());
+        idNumDV = new NumericDocValuesField("docid_intDV", 0);
         doc.add(titleDV);
+        doc.add(idNumDV);
       } else {
         titleDV = null;
+        idNumDV = null;
       }
     }
   }
@@ -244,6 +249,9 @@ public class LineFileDocs implements Clo
     final int i = id.getAndIncrement();
     docState.id.setStringValue(Integer.toString(i));
     docState.idNum.setIntValue(i);
+    if (docState.idNumDV != null) {
+      docState.idNumDV.setLongValue(i);
+    }
     return docState.doc;
   }
 }

Modified: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml?rev=1640464&r1=1640463&r2=1640464&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml (original)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml Wed Nov 19 02:09:25 2014
@@ -201,9 +201,10 @@ NOTE: Tests expect every field in this s
     <fieldtype name="str_dv_last" class="solr.StrField" stored="true" indexed="false" docValues="true" sortMissingLast="true"/>
     <fieldtype name="str_dv_first" class="solr.StrField" stored="true" indexed="false" docValues="true" sortMissingFirst="true"/>
 
-    <fieldtype name="bin" class="solr.SortableBinaryField" stored="true" indexed="true" />
-    <fieldtype name="bin_last" class="solr.SortableBinaryField" stored="true" indexed="true" sortMissingLast="true"/>
-    <fieldtype name="bin_first" class="solr.SortableBinaryField" stored="true" indexed="true" sortMissingFirst="true"/>
+    <!-- note: all 'binary' fields have docvalues, because this fieldtype doesnt support indexing -->
+    <fieldtype name="bin" class="solr.SortableBinaryField" stored="true" indexed="true" docValues="true"/>
+    <fieldtype name="bin_last" class="solr.SortableBinaryField" stored="true" indexed="true" docValues="true" sortMissingLast="true"/>
+    <fieldtype name="bin_first" class="solr.SortableBinaryField" stored="true" indexed="true" docValues="true" sortMissingFirst="true"/>
     <fieldtype name="bin_dv" class="solr.SortableBinaryField" stored="true" indexed="false" docValues="true"/>
     <fieldtype name="bin_dv_last" class="solr.SortableBinaryField" stored="true" indexed="false" docValues="true" sortMissingLast="true"/>
     <fieldtype name="bin_dv_first" class="solr.SortableBinaryField" stored="true" indexed="false" docValues="true" sortMissingFirst="true"/>