You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by eh...@apache.org on 2017/05/01 00:14:42 UTC

[2/3] lucene-solr:jira/SOLR-1485: Add {\!payload_score} qparser

Add {\!payload_score} qparser


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

Branch: refs/heads/jira/SOLR-1485
Commit: e345bef00bdd45ad735b3cf72c77aeeec61a8a8c
Parents: ced43a2
Author: Erik Hatcher <eh...@apache.org>
Authored: Sun Apr 30 20:13:16 2017 -0400
Committer: Erik Hatcher <eh...@apache.org>
Committed: Sun Apr 30 20:13:16 2017 -0400

----------------------------------------------------------------------
 .../solr/search/PayloadScoreQParserPlugin.java  | 117 +++++++++++++++++++
 .../solr/collection1/conf/schema11.xml          |   5 +-
 .../search/TestPayloadScoreQParserPlugin.java   |  54 +++++++++
 3 files changed, 172 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e345bef0/solr/core/src/java/org/apache/solr/search/PayloadScoreQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/PayloadScoreQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/PayloadScoreQParserPlugin.java
new file mode 100644
index 0000000..6013b3e
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/PayloadScoreQParserPlugin.java
@@ -0,0 +1,117 @@
+/*
+ * 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.
+ */
+
+package org.apache.solr.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.queries.payloads.PayloadFunction;
+import org.apache.lucene.queries.payloads.PayloadScoreQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.spans.SpanNearQuery;
+import org.apache.lucene.search.spans.SpanQuery;
+import org.apache.lucene.search.spans.SpanTermQuery;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.util.PayloadUtils;
+
+/**
+ * Creates a PayloadScoreQuery wrapping a SpanQuery created from the input value, applying text analysis and
+ * constructing SpanTermQuery or SpanNearQuery based on number of terms.
+ *
+ * <br>Other parameters:
+ * <br><code>f</code>, the field (required)
+ * <br><code>func</code>, payload function (min, max, or average; required)
+ * <br><code>includeSpanScore</code>, multiple payload function result by similarity score or not (default: false)
+ * <br>Example: <code>{!payload_score f=weighted_terms_dpf}Foo Bar</code> creates a SpanNearQuery with "Foo" followed by "Bar"
+ */
+public class PayloadScoreQParserPlugin extends QParserPlugin {
+  public static final String NAME = "payload_score";
+
+  @Override
+  public QParser createParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) {
+    return new QParser(qstr, localParams, params, req) {
+      @Override
+      public Query parse() throws SyntaxError {
+        String field = localParams.get(QueryParsing.F);
+        String value = localParams.get(QueryParsing.V);
+        String func = localParams.get("func");
+        boolean includeSpanScore = localParams.getBool("includeSpanScore", false);
+
+        if (field == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'f' not specified");
+        }
+
+        if (value == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "query string missing");
+        }
+
+        FieldType ft = req.getCore().getLatestSchema().getFieldType(field);
+        Analyzer analyzer = ft.getQueryAnalyzer();
+        SpanQuery query = null;
+        query = createSpanQuery(field, value, analyzer);
+
+        if (query == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "SpanQuery is null");
+        }
+
+        // note: this query(/parser) does not support func=first; 'first' is a payload() value source feature only
+        PayloadFunction payloadFunction = PayloadUtils.getPayloadFunction(func);
+        if (payloadFunction == null) throw new SyntaxError("Unknown payload function: " + func);
+
+        return new PayloadScoreQuery(query, payloadFunction, includeSpanScore);
+      }
+    };
+  }
+
+  private SpanQuery createSpanQuery(String field, String value, Analyzer analyzer) {
+    SpanQuery query;
+    try {
+      // adapted this from QueryBuilder.createSpanQuery (which isn't currently public) and added reset(), end(), and close() calls
+      TokenStream in = analyzer.tokenStream(field, value);
+      in.reset();
+
+      TermToBytesRefAttribute termAtt = in.getAttribute(TermToBytesRefAttribute.class);
+
+      List<SpanTermQuery> terms = new ArrayList<>();
+      while (in.incrementToken()) {
+        terms.add(new SpanTermQuery(new Term(field, termAtt.getBytesRef())));
+      }
+      in.end();
+      in.close();
+
+      if (terms.isEmpty()) {
+        query = null;
+      } else if (terms.size() == 1) {
+        query = terms.get(0);
+      } else {
+        query = new SpanNearQuery(terms.toArray(new SpanTermQuery[terms.size()]), 0, true);
+      }
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+    }
+    return query;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e345bef0/solr/core/src/test-files/solr/collection1/conf/schema11.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema11.xml b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
index b91c680..6482b37 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema11.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
@@ -451,13 +451,10 @@ valued. -->
   <!-- For testing payload function -->
   <dynamicField name="*_dp" type="delimited_payloads" indexed="true" stored="true" multiValued="false"/>
   <fieldType name="delimited_payloads" class="solr.TextField" positionIncrementGap="100">
-    <analyzer type="index">
+    <analyzer>
       <tokenizer class="solr.MockTokenizerFactory"/>
       <filter class="solr.DelimitedPayloadTokenFilterFactory" encoder="float"/>
     </analyzer>
-    <analyzer type="query">
-      <tokenizer class="solr.MockTokenizerFactory" pattern="keyword"/>
-    </analyzer>
   </fieldType>
 
   <dynamicField name="*_extf" type="file"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e345bef0/solr/core/src/test/org/apache/solr/search/TestPayloadScoreQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestPayloadScoreQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestPayloadScoreQParserPlugin.java
new file mode 100644
index 0000000..10945b6
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/TestPayloadScoreQParserPlugin.java
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+package org.apache.solr.search;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestPayloadScoreQParserPlugin extends SolrTestCaseJ4 {
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig.xml", "schema11.xml");
+    createIndex();
+  }
+
+  public static void createIndex() {
+    assertU(adoc("id","1", "vals_dp","A|1.0 B|2.0 C|3.0 mult|50 mult|100 x|22 x|37 x|19"));
+    assertU(commit());
+  }
+
+  @Test
+  public void test() {
+    clearIndex();
+
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dp v=B func=min}"), "//float[@name='score']='2.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dp v=mult func=min}"), "//float[@name='score']='50.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dp v=mult func=max}"), "//float[@name='score']='100.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dp v=mult func=average}"), "//float[@name='score']='75.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dp func=min}A B"), "//float[@name='score']='1.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dp func=min}B C"), "//float[@name='score']='2.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dp func=max}B C"), "//float[@name='score']='3.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dp func=average}B C"), "//float[@name='score']='2.5'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dp func=max}A B C"), "//float[@name='score']='3.0'");
+
+    // TODO: fix this includeSpanScore test to be less brittle - score result is score of "A" (via BM25) multipled by 1.0 (payload value)
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dp v=A func=min}"), "//float[@name='score']='1.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dp v=A func=min includeSpanScore=true}"), "//float[@name='score']='0.25811607'");
+  }
+}