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 2017/02/09 11:18:43 UTC

[2/2] lucene-solr:branch_6x: LUCENE-7679: MemoryIndex should take into account omitNorms

LUCENE-7679: MemoryIndex should take into account omitNorms


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/cecc44c7
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/cecc44c7
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/cecc44c7

Branch: refs/heads/branch_6x
Commit: cecc44c731b18c46bf2c0eaa92732eb207daa6ca
Parents: 05e0250
Author: Alan Woodward <ro...@apache.org>
Authored: Tue Feb 7 10:41:20 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Feb 9 10:47:18 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 +
 .../java/org/apache/lucene/index/FieldInfo.java |  3 +-
 .../apache/lucene/index/memory/MemoryIndex.java | 75 ++++++++++++--------
 .../lucene/index/memory/TestMemoryIndex.java    | 18 ++++-
 4 files changed, 67 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cecc44c7/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 231bc9f..971ea54 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -57,6 +57,9 @@ Bug Fixes
 * LUCENE-7676: Fixed FilterCodecReader to override more super-class methods.
   Also added TestFilterCodecReader class. (Christine Poerschke)
 
+* LUCENE-7679: MemoryIndex was ignoring omitNorms settings on passed-in
+  IndexableFields. (Alan Woodward)
+
 Improvements
 
 * LUCENE-7055: Added Weight#scorerSupplier, which allows to estimate the cost

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cecc44c7/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
index 4e02320..422292b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
@@ -203,7 +203,8 @@ public final class FieldInfo {
     return pointNumBytes;
   }
 
-  void setDocValuesType(DocValuesType type) {
+  /** Record that this field is indexed with docvalues, with the specified type */
+  public void setDocValuesType(DocValuesType type) {
     if (type == null) {
       throw new NullPointerException("DocValuesType must not be null (field: \"" + name + "\")");
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cecc44c7/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
index 362e7fb..42281a2 100644
--- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
+++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
@@ -33,6 +33,7 @@ import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.*;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
@@ -194,6 +195,8 @@ public class MemoryIndex {
 
   private Similarity normSimilarity = IndexSearcher.getDefaultSimilarity();
 
+  private FieldType defaultFieldType = new FieldType();
+
   /**
    * Constructs an empty instance that will not store offsets or payloads.
    */
@@ -235,6 +238,9 @@ public class MemoryIndex {
   MemoryIndex(boolean storeOffsets, boolean storePayloads, long maxReusedBytes) {
     this.storeOffsets = storeOffsets;
     this.storePayloads = storePayloads;
+    this.defaultFieldType.setIndexOptions(storeOffsets ?
+        IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+    this.defaultFieldType.setStoreTermVectors(true);
     this.bytesUsed = Counter.newCounter();
     final int maxBufferedByteBlocks = (int)((maxReusedBytes/2) / ByteBlockPool.BYTE_BLOCK_SIZE );
     final int maxBufferedIntBlocks = (int) ((maxReusedBytes - (maxBufferedByteBlocks*ByteBlockPool.BYTE_BLOCK_SIZE))/(IntBlockPool.INT_BLOCK_SIZE * Integer.BYTES));
@@ -268,8 +274,8 @@ public class MemoryIndex {
       throw new IllegalArgumentException("analyzer must not be null");
     
     TokenStream stream = analyzer.tokenStream(fieldName, text);
-    addField(fieldName, stream, 1.0f, analyzer.getPositionIncrementGap(fieldName), analyzer.getOffsetGap(fieldName),
-        DocValuesType.NONE, null, 0, 0, null);
+    storeTerms(getInfo(fieldName, defaultFieldType), stream, 1.0f,
+        analyzer.getPositionIncrementGap(fieldName), analyzer.getOffsetGap(fieldName));
   }
 
   /**
@@ -384,10 +390,11 @@ public class MemoryIndex {
    * @param field the field to add
    * @param analyzer the analyzer to use for term analysis
    * @param boost a field boost
-   * @throws IllegalArgumentException if the field is a DocValues or Point field, as these
-   *                                  structures are not supported by MemoryIndex
    */
   public void addField(IndexableField field, Analyzer analyzer, float boost) {
+
+    Info info = getInfo(field.name(), field.fieldType());
+
     int offsetGap;
     TokenStream tokenStream;
     int positionIncrementGap;
@@ -400,6 +407,9 @@ public class MemoryIndex {
       tokenStream = field.tokenStream(null, null);
       positionIncrementGap = 0;
     }
+    if (tokenStream != null) {
+      storeTerms(info, tokenStream, boost, positionIncrementGap, offsetGap);
+    }
 
     DocValuesType docValuesType = field.fieldType().docValuesType();
     Object docValuesValue;
@@ -419,12 +429,14 @@ public class MemoryIndex {
       default:
         throw new UnsupportedOperationException("unknown doc values type [" + docValuesType + "]");
     }
-    BytesRef pointValue = null;
+    if (docValuesValue != null) {
+      storeDocValues(info, docValuesType, docValuesValue);
+    }
+
     if (field.fieldType().pointDimensionCount() > 0) {
-      pointValue = field.binaryValue();
+      storePointValues(info, field.binaryValue());
     }
-    addField(field.name(), tokenStream, boost, positionIncrementGap, offsetGap, docValuesType, docValuesValue,
-        field.fieldType().pointDimensionCount(), field.fieldType().pointNumBytes(), pointValue);
+
   }
   
   /**
@@ -493,42 +505,40 @@ public class MemoryIndex {
    * @see org.apache.lucene.document.Field#setBoost(float)
    */
   public void addField(String fieldName, TokenStream tokenStream, float boost, int positionIncrementGap, int offsetGap) {
-    addField(fieldName, tokenStream, boost, positionIncrementGap, offsetGap, DocValuesType.NONE, null, 0, 0, null);
+    Info info = getInfo(fieldName, defaultFieldType);
+    storeTerms(info, tokenStream, boost, positionIncrementGap, offsetGap);
   }
 
-  private void addField(String fieldName, TokenStream tokenStream, float boost, int positionIncrementGap, int offsetGap,
-                        DocValuesType docValuesType, Object docValuesValue, int pointDimensionCount, int pointNumBytes,
-                        BytesRef pointValue) {
-
+  private Info getInfo(String fieldName, IndexableFieldType fieldType) {
     if (frozen) {
       throw new IllegalArgumentException("Cannot call addField() when MemoryIndex is frozen");
     }
     if (fieldName == null) {
       throw new IllegalArgumentException("fieldName must not be null");
     }
-    if (boost <= 0.0f) {
-      throw new IllegalArgumentException("boost factor must be greater than 0.0");
-    }
-
     Info info = fields.get(fieldName);
     if (info == null) {
-      IndexOptions indexOptions = storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
-      FieldInfo fieldInfo = new FieldInfo(fieldName, fields.size(), true, false, storePayloads, indexOptions, docValuesType, -1, Collections.emptyMap(), 0, 0);
-      fields.put(fieldName, info = new Info(fieldInfo, byteBlockPool));
+      fields.put(fieldName, info = new Info(createFieldInfo(fieldName, fields.size(), fieldType), byteBlockPool));
     }
-    if (pointDimensionCount > 0) {
-      storePointValues(info, pointDimensionCount, pointNumBytes, pointValue);
+    if (fieldType.pointDimensionCount() != info.fieldInfo.getPointDimensionCount()) {
+      if (fieldType.pointDimensionCount() > 0)
+        info.fieldInfo.setPointDimensions(fieldType.pointDimensionCount(), fieldType.pointNumBytes());
     }
-    if (docValuesType != DocValuesType.NONE) {
-      storeDocValues(info, docValuesType, docValuesValue);
-    }
-    if (tokenStream != null) {
-      storeTerms(info, tokenStream, boost, positionIncrementGap, offsetGap);
+    if (fieldType.docValuesType() != info.fieldInfo.getDocValuesType()) {
+      if (fieldType.docValuesType() != DocValuesType.NONE)
+        info.fieldInfo.setDocValuesType(fieldType.docValuesType());
     }
+    return info;
+  }
+
+  private FieldInfo createFieldInfo(String fieldName, int ord, IndexableFieldType fieldType) {
+    IndexOptions indexOptions = storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+    return new FieldInfo(fieldName, ord, fieldType.storeTermVectors(), fieldType.omitNorms(), storePayloads,
+        indexOptions, fieldType.docValuesType(), -1, Collections.emptyMap(),
+        fieldType.pointDimensionCount(), fieldType.pointNumBytes());
   }
 
-  private void storePointValues(Info info, int pointDimensionCount, int pointNumBytes, BytesRef pointValue) {
-    info.fieldInfo.setPointDimensions(pointDimensionCount, pointNumBytes);
+  private void storePointValues(Info info, BytesRef pointValue) {
     if (info.pointValues == null) {
       info.pointValues = new BytesRef[4];
     }
@@ -590,6 +600,11 @@ public class MemoryIndex {
   }
 
   private void storeTerms(Info info, TokenStream tokenStream, float boost, int positionIncrementGap, int offsetGap) {
+
+    if (boost <= 0.0f) {
+      throw new IllegalArgumentException("boost factor must be greater than 0.0");
+    }
+
     int pos = -1;
     int offset = 0;
     if (info.numTokens == 0) {
@@ -1625,7 +1640,7 @@ public class MemoryIndex {
     @Override
     public NumericDocValues getNormValues(String field) {
       Info info = fields.get(field);
-      if (info == null) {
+      if (info == null || info.fieldInfo.omitsNorms()) {
         return null;
       }
       return info.getNormDocValues();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cecc44c7/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
index 1d7c60e..75344e3 100644
--- a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
+++ b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
@@ -45,6 +45,7 @@ import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.LeafReader;
@@ -196,6 +197,21 @@ public class TestMemoryIndex extends LuceneTestCase {
   }
 
   @Test
+  public void testOmitNorms() throws IOException {
+    MemoryIndex mi = new MemoryIndex();
+    FieldType ft = new FieldType();
+    ft.setTokenized(true);
+    ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+    ft.setOmitNorms(true);
+    mi.addField(new Field("f1", "some text in here", ft), analyzer);
+    mi.freeze();
+
+    LeafReader leader = (LeafReader) mi.createSearcher().getIndexReader();
+    NumericDocValues norms = leader.getNormValues("f1");
+    assertNull(norms);
+  }
+
+  @Test
   public void testBuildFromDocument() {
 
     Document doc = new Document();
@@ -269,7 +285,7 @@ public class TestMemoryIndex extends LuceneTestCase {
     try {
       MemoryIndex.fromDocument(doc, analyzer);
     } catch (IllegalArgumentException e) {
-      assertEquals("Can't add [BINARY] doc values field [field], because [NUMERIC] doc values field already exists", e.getMessage());
+      assertEquals("cannot change DocValues type from NUMERIC to BINARY for field \"field\"", e.getMessage());
     }
 
     doc = new Document();