You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2013/10/19 21:01:08 UTC

svn commit: r1533808 - in /lucene/dev/trunk/lucene: ./ suggest/ suggest/src/java/org/apache/lucene/search/suggest/ suggest/src/test/org/apache/lucene/search/suggest/

Author: mikemccand
Date: Sat Oct 19 19:01:07 2013
New Revision: 1533808

URL: http://svn.apache.org/r1533808
Log:
LUCENE-4998: add DocumentExpressionDictionary, to compute each suggestion's weight using a javascript expression

Added:
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentExpressionDictionary.java   (with props)
    lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentExpressionDictionaryTest.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/suggest/build.xml
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1533808&r1=1533807&r2=1533808&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Sat Oct 19 19:01:07 2013
@@ -113,6 +113,10 @@ New Features
 
 * LUCENE-5269: Add CodepointCountFilter. (Robert Muir)
 
+* LUCENE-5294: Suggest module: add DocumentExpressionDictionary to
+  compute each suggestion's weight using a javascript expression.
+  (Areek Zillur via Mike McCandless)
+
 Bug Fixes
 
 * LUCENE-4998: Fixed a few places to pass IOContext.READONCE instead

Modified: lucene/dev/trunk/lucene/suggest/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/build.xml?rev=1533808&r1=1533807&r2=1533808&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/build.xml (original)
+++ lucene/dev/trunk/lucene/suggest/build.xml Sat Oct 19 19:01:07 2013
@@ -31,6 +31,9 @@
   <path id="classpath">
     <pathelement path="${analyzers-common.jar}"/>
     <pathelement path="${misc.jar}"/>
+    <pathelement path="${expressions.jar}"/>
+    <pathelement path="${queries.jar}"/>
+    <fileset dir="${common.dir}/expressions/lib"/>
     <path refid="base.classpath"/>
   </path>
 
@@ -43,6 +46,6 @@
     </invoke-module-javadoc>
   </target>
 
-  <target name="compile-core" depends="jar-misc, jar-analyzers-common, common.compile-core" />
+  <target name="compile-core" depends="jar-expressions, jar-misc, jar-analyzers-common, common.compile-core" />
 
 </project>

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java?rev=1533808&r1=1533807&r2=1533808&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java Sat Oct 19 19:01:07 2013
@@ -45,10 +45,13 @@ import org.apache.lucene.util.BytesRefIt
  */
 public class DocumentDictionary implements Dictionary {
   
-  private final IndexReader reader;
+  /** {@link IndexReader} to load documents from */
+  protected final IndexReader reader;
+
+  /** Field to read payload from */
+  protected final String payloadField;
   private final String field;
   private final String weightField;
-  private final String payloadField;
   
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
@@ -79,8 +82,9 @@ public class DocumentDictionary implemen
   public BytesRefIterator getWordsIterator() throws IOException {
     return new DocumentInputIterator(payloadField!=null);
   }
-    
-  final class DocumentInputIterator implements InputIterator {
+
+  /** Implements {@link InputIterator} from stored fields. */
+  protected class DocumentInputIterator implements InputIterator {
     private final int docCount;
     private final Set<String> relevantFields;
     private final boolean hasPayloads;
@@ -88,6 +92,7 @@ public class DocumentDictionary implemen
     private int currentDocId = -1;
     private long currentWeight;
     private BytesRef currentPayload;
+    private StoredDocument doc;
     
     /**
      * Creates an iterator over term, weight and payload fields from the lucene
@@ -99,13 +104,7 @@ public class DocumentDictionary implemen
       this.hasPayloads = hasPayloads;
       currentPayload = null;
       liveDocs = MultiFields.getLiveDocs(reader);
-      List<String> relevantFieldList;
-      if(hasPayloads) {
-        relevantFieldList = Arrays.asList(field, weightField, payloadField);
-      } else {
-        relevantFieldList = Arrays.asList(field, weightField);
-      }
-      this.relevantFields = new HashSet<>(relevantFieldList);
+      this.relevantFields = getRelevantFields(new String [] {field, weightField, payloadField});
     }
 
     @Override
@@ -121,7 +120,7 @@ public class DocumentDictionary implemen
           continue;
         }
 
-        StoredDocument doc = reader.document(currentDocId, relevantFields);
+        doc = reader.document(currentDocId, relevantFields);
         
         if (hasPayloads) {
           StorableField payload = doc.getField(payloadField);
@@ -133,13 +132,7 @@ public class DocumentDictionary implemen
           currentPayload = payload.binaryValue();
         }
         
-        StorableField weight = doc.getField(weightField);
-        if (weight == null) {
-          throw new IllegalArgumentException(weightField + " does not exist");
-        } else if (weight.numericValue() == null) {
-          throw new IllegalArgumentException(weightField + " does not have numeric value");
-        }
-        currentWeight = weight.numericValue().longValue();
+        currentWeight = getWeight(currentDocId);
         
         StorableField fieldVal = doc.getField(field);
         if (fieldVal == null) {
@@ -162,6 +155,26 @@ public class DocumentDictionary implemen
     public boolean hasPayloads() {
       return hasPayloads;
     }
+
+    /** Return the suggestion weight for this document */
+    protected long getWeight(int docId) {
+      StorableField weight = doc.getField(weightField);
+      if (weight == null) {
+        throw new IllegalArgumentException(weightField + " does not exist");
+      } else if (weight.numericValue() == null) {
+        throw new IllegalArgumentException(weightField + " does not have numeric value");
+      }
+      return weight.numericValue().longValue();
+    }
     
+    private Set<String> getRelevantFields(String... fields) {
+      Set<String> relevantFields = new HashSet<>();
+      for (String relevantField : fields) {
+        if (relevantField != null) {
+          relevantFields.add(relevantField);
+        }
+      }
+      return relevantFields;
+    }
   }
 }

Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentExpressionDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentExpressionDictionary.java?rev=1533808&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentExpressionDictionary.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentExpressionDictionary.java Sat Oct 19 19:01:07 2013
@@ -0,0 +1,121 @@
+package org.apache.lucene.search.suggest;
+
+/*
+ * 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 java.io.IOException;
+import java.text.ParseException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.lucene.document.NumericDocValuesField; // javadocs
+import org.apache.lucene.expressions.Expression;
+import org.apache.lucene.expressions.SimpleBindings;
+import org.apache.lucene.expressions.js.JavascriptCompiler;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.CompositeReader;  // javadocs
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.util.BytesRefIterator;
+
+
+/**
+ * Dictionary with terms and optionally payload information 
+ * taken from stored fields in a Lucene index. Similar to 
+ * {@link DocumentDictionary}, except it computes the weight
+ * of the terms in a document based on a user-defined expression
+ * having one or more {@link NumericDocValuesField} in the document.
+ * 
+ * <b>NOTE:</b> 
+ *  <ul>
+ *    <li>
+ *      The term and (optionally) payload fields supplied
+ *      are required for ALL documents and has to be stored
+ *    </li>
+ *    <li>
+ *      {@link CompositeReader} is not supported.
+ *    </li>
+ *  </ul>
+ */
+public class DocumentExpressionDictionary extends DocumentDictionary {
+  
+  private ValueSource weightsValueSource;
+  
+  /**
+   * Creates a new dictionary with the contents of the fields named <code>field</code>
+   * for the terms and computes the corresponding weights of the term by compiling the
+   * user-defined <code>weightExpression</code> using the <code>sortFields</code>
+   * bindings.
+   */
+  public DocumentExpressionDictionary(IndexReader reader, String field,
+      String weightExpression, Set<SortField> sortFields) {
+    this(reader, field, weightExpression, sortFields, null);
+  }
+  
+  /**
+   * Creates a new dictionary with the contents of the fields named <code>field</code>
+   * for the terms, <code>payloadField</code> for the corresponding payloads
+   * and computes the corresponding weights of the term by compiling the
+   * user-defined <code>weightExpression</code> using the <code>sortFields</code>
+   * bindings.
+   */
+  public DocumentExpressionDictionary(IndexReader reader, String field,
+      String weightExpression, Set<SortField> sortFields, String payload) {
+    super(reader, field, null, payload);
+    Expression expression = null;
+    try {
+      expression = JavascriptCompiler.compile(weightExpression);
+    } catch (ParseException e) {
+      throw new RuntimeException();
+    }
+    SimpleBindings bindings = new SimpleBindings();
+    for (SortField sortField: sortFields) {
+      bindings.add(sortField);
+    }
+    weightsValueSource = expression.getValueSource(bindings);
+    
+  }
+  
+  @Override
+  public BytesRefIterator getWordsIterator() throws IOException {
+    return new DocumentExpressionInputIterator(payloadField!=null);
+  }
+  
+  final class DocumentExpressionInputIterator extends DocumentDictionary.DocumentInputIterator {
+    
+    private FunctionValues weightValues;
+    
+    public DocumentExpressionInputIterator(boolean hasPayloads)
+        throws IOException {
+      super(hasPayloads);
+      List<AtomicReaderContext> leaves = reader.leaves();
+      if (leaves.size() > 1) {
+        throw new IllegalArgumentException("CompositeReader is not supported");
+      }
+      weightValues = weightsValueSource.getValues(new HashMap<String, Object>(), leaves.get(0));
+    }
+    
+    @Override
+    protected long getWeight(int docId) {
+      return weightValues.longVal(docId);
+    }
+
+  }
+}

Added: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentExpressionDictionaryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentExpressionDictionaryTest.java?rev=1533808&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentExpressionDictionaryTest.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentExpressionDictionaryTest.java Sat Oct 19 19:01:07 2013
@@ -0,0 +1,195 @@
+package org.apache.lucene.search.suggest;
+
+/*
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.spell.Dictionary;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.Test;
+
+public class DocumentExpressionDictionaryTest extends LuceneTestCase {
+  
+  static final String FIELD_NAME = "f1";
+  static final String WEIGHT_FIELD_NAME_1 = "w1";
+  static final String WEIGHT_FIELD_NAME_2 = "w2";
+  static final String WEIGHT_FIELD_NAME_3 = "w3";
+  static final String PAYLOAD_FIELD_NAME = "p1";
+  
+  private Map<String, Document> generateIndexDocuments(int ndocs) {
+    Map<String, Document> docs = new HashMap<>();
+    for(int i = 0; i < ndocs ; i++) {
+      Field field = new TextField(FIELD_NAME, "field_" + i, Field.Store.YES);
+      Field payload = new StoredField(PAYLOAD_FIELD_NAME, new BytesRef("payload_" + i));
+      Field weight1 = new NumericDocValuesField(WEIGHT_FIELD_NAME_1, 10 + i);
+      Field weight2 = new NumericDocValuesField(WEIGHT_FIELD_NAME_2, 20 + i);
+      Field weight3 = new NumericDocValuesField(WEIGHT_FIELD_NAME_3, 30 + i);
+      Document doc = new Document();
+      doc.add(field);
+      doc.add(payload);
+      doc.add(weight1);
+      doc.add(weight2);
+      doc.add(weight3);
+      docs.put(field.stringValue(), doc);
+    }
+    return docs;
+  }
+  
+  @Test
+  public void testBasic() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(10);
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+    IndexReader ir = DirectoryReader.open(dir);
+    Set<SortField> sortFields = new HashSet<SortField>(); 
+    sortFields.add(new SortField(WEIGHT_FIELD_NAME_1, SortField.Type.LONG));
+    sortFields.add(new SortField(WEIGHT_FIELD_NAME_2, SortField.Type.LONG));
+    sortFields.add(new SortField(WEIGHT_FIELD_NAME_3, SortField.Type.LONG));
+    Dictionary dictionary = new DocumentExpressionDictionary(ir, FIELD_NAME, "((w1 + w2) - w3)", sortFields, PAYLOAD_FIELD_NAME);
+    InputIterator tfp = (InputIterator) dictionary.getWordsIterator();
+    BytesRef f;
+    while((f = tfp.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      long w3 = doc.getField(WEIGHT_FIELD_NAME_3).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(tfp.weight(), (w1 + w2) - w3);
+      assertTrue(tfp.payload().equals(doc.getField(PAYLOAD_FIELD_NAME).binaryValue()));
+    }
+    assertTrue(docs.isEmpty());
+    ir.close();
+    dir.close();
+  }
+
+  @Test
+  public void testWithoutPayload() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(10);
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+    IndexReader ir = DirectoryReader.open(dir);
+    Set<SortField> sortFields = new HashSet<SortField>(); 
+    sortFields.add(new SortField(WEIGHT_FIELD_NAME_1, SortField.Type.LONG));
+    sortFields.add(new SortField(WEIGHT_FIELD_NAME_2, SortField.Type.LONG));
+    sortFields.add(new SortField(WEIGHT_FIELD_NAME_3, SortField.Type.LONG));
+    Dictionary dictionary = new DocumentExpressionDictionary(ir, FIELD_NAME, "w1 + (0.2 * w2) - (w3 - w1)/2", sortFields);
+    InputIterator tfp = (InputIterator) dictionary.getWordsIterator();
+    BytesRef f;
+    while((f = tfp.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      long w3 = doc.getField(WEIGHT_FIELD_NAME_3).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(tfp.weight(), (long)(w1 + (0.2 * w2) - (w3 - w1)/2));
+      assertEquals(tfp.payload(), null);
+    }
+    assertTrue(docs.isEmpty());
+    ir.close();
+    dir.close();
+  }
+  
+  @Test
+  public void testWithDeletions() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(10);
+    Random rand = random();
+    List<String> termsToDel = new ArrayList<>();
+    for(Document doc : docs.values()) {
+      if(rand.nextBoolean()) {
+        termsToDel.add(doc.get(FIELD_NAME));
+      }
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    
+    Term[] delTerms = new Term[termsToDel.size()];
+    for(int i=0; i < termsToDel.size() ; i++) {
+      delTerms[i] = new Term(FIELD_NAME, termsToDel.get(i));
+    }
+    
+    for(Term delTerm: delTerms) {
+      writer.deleteDocuments(delTerm);  
+    }
+    writer.commit();
+    writer.close();
+    
+    for(String termToDel: termsToDel) {
+      assertTrue(null!=docs.remove(termToDel));
+    }
+    
+    IndexReader ir = DirectoryReader.open(dir);
+    assertEquals(ir.numDocs(), docs.size());
+    Set<SortField> sortFields = new HashSet<SortField>(); 
+    sortFields.add(new SortField(WEIGHT_FIELD_NAME_1, SortField.Type.LONG));
+    sortFields.add(new SortField(WEIGHT_FIELD_NAME_2, SortField.Type.LONG));
+    Dictionary dictionary = new DocumentExpressionDictionary(ir, FIELD_NAME, "w2-w1", sortFields, PAYLOAD_FIELD_NAME);
+    InputIterator tfp = (InputIterator) dictionary.getWordsIterator();
+    BytesRef f;
+    while((f = tfp.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(tfp.weight(), w2-w1);
+      assertTrue(tfp.payload().equals(doc.getField(PAYLOAD_FIELD_NAME).binaryValue()));
+    }
+    assertTrue(docs.isEmpty());
+    ir.close();
+    dir.close();
+  }
+  
+}