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/02 01:35:38 UTC

lucene-solr:master: SOLR-1485: Add payload support

Repository: lucene-solr
Updated Branches:
  refs/heads/master ee8ce57e5 -> 6c565c001


SOLR-1485: Add payload support


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

Branch: refs/heads/master
Commit: 6c565c001bb48af0c37a4d4909ba2f98d51e7fe6
Parents: ee8ce57
Author: Erik Hatcher <eh...@apache.org>
Authored: Mon May 1 21:35:29 2017 -0400
Committer: Erik Hatcher <eh...@apache.org>
Committed: Mon May 1 21:35:29 2017 -0400

----------------------------------------------------------------------
 .../queries/payloads/PayloadScoreQuery.java     |  13 +-
 .../queries/payloads/SpanPayloadCheckQuery.java |   2 +-
 solr/CHANGES.txt                                |   3 +
 .../solr/search/FloatPayloadValueSource.java    | 221 +++++++++++++++++++
 .../solr/search/PayloadCheckQParserPlugin.java  | 103 +++++++++
 .../solr/search/PayloadScoreQParserPlugin.java  |  78 +++++++
 .../org/apache/solr/search/QParserPlugin.java   |   2 +
 .../apache/solr/search/ValueSourceParser.java   |  50 ++++-
 .../PayloadScoringSimilarityWrapper.java        |  80 +++++++
 .../similarities/SchemaSimilarityFactory.java   |  23 +-
 .../org/apache/solr/util/PayloadDecoder.java    |  28 +++
 .../java/org/apache/solr/util/PayloadUtils.java | 134 +++++++++++
 .../solr/collection1/conf/schema11.xml          |  32 ++-
 .../solr/collection1/conf/schema15.xml          |   8 +
 .../apache/solr/search/QueryEqualityTest.java   |  44 +++-
 .../search/TestPayloadCheckQParserPlugin.java   |  73 ++++++
 .../search/TestPayloadScoreQParserPlugin.java   |  54 +++++
 .../solr/search/function/TestFunctionQuery.java |  30 ++-
 .../conf/managed-schema                         |  25 +++
 19 files changed, 986 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java
index 43d33fd..3db80fb 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java
@@ -41,8 +41,7 @@ import org.apache.lucene.search.spans.Spans;
 import org.apache.lucene.util.BytesRef;
 
 /**
- * A Query class that uses a {@link PayloadFunction} to modify the score of a
- * wrapped SpanQuery
+ * A Query class that uses a {@link PayloadFunction} to modify the score of a wrapped SpanQuery
  *
  * NOTE: In order to take advantage of this with the default scoring implementation
  * ({@link ClassicSimilarity}), you must override {@link ClassicSimilarity#scorePayload(int, int, int, BytesRef)},
@@ -94,7 +93,15 @@ public class PayloadScoreQuery extends SpanQuery {
 
   @Override
   public String toString(String field) {
-    return "PayloadScoreQuery[" + wrappedQuery.toString(field) + "; " + function.getClass().getSimpleName() + "; " + includeSpanScore + "]";
+    StringBuilder buffer = new StringBuilder();
+    buffer.append("PayloadScoreQuery(");
+    buffer.append(wrappedQuery.toString(field));
+    buffer.append(", function: ");
+    buffer.append(function.getClass().getSimpleName());
+    buffer.append(", includeSpanScore: ");
+    buffer.append(includeSpanScore);
+    buffer.append(")");
+    return buffer.toString();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java
index ca0e062..29f3b4a 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java
@@ -173,7 +173,7 @@ public class SpanPayloadCheckQuery extends SpanQuery {
   @Override
   public String toString(String field) {
     StringBuilder buffer = new StringBuilder();
-    buffer.append("spanPayCheck(");
+    buffer.append("SpanPayloadCheckQuery(");
     buffer.append(match.toString(field));
     buffer.append(", payloadRef: ");
     for (BytesRef bytes : payloadToMatch) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 81288d8..31f9d24 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -199,6 +199,9 @@ New Features
 * SOLR-9596: Add Solr support for SimpleTextCodec, via <codecFactory class="solr.SimpleTextCodecFactory"/>
   in solrconfig.xml (per-field specification in the schema is not possible). (Steve Rowe)
 
+* SOLR-1485: Add payload support with payload() value source and {!payload_score} and {!payload_check}
+  query parsers.  (Erik Hatcher)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/core/src/java/org/apache/solr/search/FloatPayloadValueSource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/FloatPayloadValueSource.java b/solr/core/src/java/org/apache/solr/search/FloatPayloadValueSource.java
new file mode 100644
index 0000000..e926ef8
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/FloatPayloadValueSource.java
@@ -0,0 +1,221 @@
+/*
+ * 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.Map;
+
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.FloatDocValues;
+import org.apache.lucene.queries.payloads.PayloadFunction;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.util.PayloadDecoder;
+
+public class FloatPayloadValueSource extends ValueSource {
+  protected final String field;
+  protected final String val;
+  protected final String indexedField;
+  protected final BytesRef indexedBytes;
+  protected final PayloadDecoder decoder;
+  protected final PayloadFunction payloadFunction;
+  protected final ValueSource defaultValueSource;
+
+  public FloatPayloadValueSource(String field, String val, String indexedField, BytesRef indexedBytes,
+                                 PayloadDecoder decoder, PayloadFunction payloadFunction, ValueSource defaultValueSource) {
+    this.field = field;
+    this.val = val;
+    this.indexedField = indexedField;
+    this.indexedBytes = indexedBytes;
+    this.decoder = decoder;
+    this.payloadFunction = payloadFunction;
+    this.defaultValueSource = defaultValueSource;
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+
+    Fields fields = readerContext.reader().fields();
+    final Terms terms = fields.terms(indexedField);
+
+    FunctionValues defaultValues = defaultValueSource.getValues(context, readerContext);
+
+    // copied the bulk of this from TFValueSource - TODO: this is a very repeated pattern - base-class this advance logic stuff?
+    return new FloatDocValues(this) {
+      PostingsEnum docs ;
+      int atDoc;
+      int lastDocRequested = -1;
+
+      { reset(); }
+
+      public void reset() throws IOException {
+        // no one should call us for deleted docs?
+
+        if (terms != null) {
+          final TermsEnum termsEnum = terms.iterator();
+          if (termsEnum.seekExact(indexedBytes)) {
+            docs = termsEnum.postings(null, PostingsEnum.ALL);
+          } else {
+            docs = null;
+          }
+        } else {
+          docs = null;
+        }
+
+        if (docs == null) {
+          // dummy PostingsEnum so floatVal() can work
+          // when would this be called?  if field/val did not match?  this is called for every doc?  create once and cache?
+          docs = new PostingsEnum() {
+            @Override
+            public int freq() {
+              return 0;
+            }
+
+            @Override
+            public int nextPosition() throws IOException {
+              return -1;
+            }
+
+            @Override
+            public int startOffset() throws IOException {
+              return -1;
+            }
+
+            @Override
+            public int endOffset() throws IOException {
+              return -1;
+            }
+
+            @Override
+            public BytesRef getPayload() throws IOException {
+              return null;
+            }
+
+            @Override
+            public int docID() {
+              return DocIdSetIterator.NO_MORE_DOCS;
+            }
+
+            @Override
+            public int nextDoc() {
+              return DocIdSetIterator.NO_MORE_DOCS;
+            }
+
+            @Override
+            public int advance(int target) {
+              return DocIdSetIterator.NO_MORE_DOCS;
+            }
+
+            @Override
+            public long cost() {
+              return 0;
+            }
+          };
+        }
+        atDoc = -1;
+      }
+
+      @Override
+      public float floatVal(int doc) {
+        try {
+          if (doc < lastDocRequested) {
+            // out-of-order access.... reset
+            reset();
+          }
+          lastDocRequested = doc;
+
+          if (atDoc < doc) {
+            atDoc = docs.advance(doc);
+          }
+
+          if (atDoc > doc) {
+            // term doesn't match this document... either because we hit the
+            // end, or because the next doc is after this doc.
+            return defaultValues.floatVal(doc);
+          }
+
+          // a match!
+          int freq = docs.freq();
+          int numPayloadsSeen = 0;
+          float currentScore = 0;
+          for (int i=0; i < freq; i++) {
+            docs.nextPosition();
+            BytesRef payload = docs.getPayload();
+            if (payload != null) {
+              float payloadVal = decoder.decode(null, atDoc, docs.startOffset(), docs.endOffset(), payload);
+
+              // payloadFunction = null represents "first"
+              if (payloadFunction == null) return payloadVal;
+
+              currentScore = payloadFunction.currentScore(doc, indexedField, docs.startOffset(), docs.endOffset(),
+                  numPayloadsSeen, currentScore, payloadVal);
+              numPayloadsSeen++;
+
+            }
+          }
+
+          return (numPayloadsSeen > 0) ? payloadFunction.docScore(doc, indexedField, numPayloadsSeen, currentScore) : defaultValues.floatVal(doc);
+        } catch (IOException e) {
+          throw new RuntimeException("caught exception in function "+description()+" : doc="+doc, e);
+        }
+      }
+    };
+  }
+
+  // TODO: should this be formalized at the ValueSource level?  Seems to be the convention
+  public String name() {
+    return "payload";
+  }
+
+  @Override
+  public String description() {
+    return name() + '(' + field + ',' + val + ',' + defaultValueSource.toString() + ')';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    FloatPayloadValueSource that = (FloatPayloadValueSource) o;
+
+    if (!indexedField.equals(that.indexedField)) return false;
+    if (indexedBytes != null ? !indexedBytes.equals(that.indexedBytes) : that.indexedBytes != null) return false;
+    if (!decoder.equals(that.decoder)) return false;
+    if (payloadFunction != null ? !payloadFunction.equals(that.payloadFunction) : that.payloadFunction != null)
+      return false;
+    return defaultValueSource.equals(that.defaultValueSource);
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = indexedField.hashCode();
+    result = 31 * result + (indexedBytes != null ? indexedBytes.hashCode() : 0);
+    result = 31 * result + decoder.hashCode();
+    result = 31 * result + (payloadFunction != null ? payloadFunction.hashCode() : 0);
+    result = 31 * result + defaultValueSource.hashCode();
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/core/src/java/org/apache/solr/search/PayloadCheckQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/PayloadCheckQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/PayloadCheckQParserPlugin.java
new file mode 100644
index 0000000..83090bf
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/PayloadCheckQParserPlugin.java
@@ -0,0 +1,103 @@
+/*
+ * 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.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.payloads.FloatEncoder;
+import org.apache.lucene.analysis.payloads.IdentityEncoder;
+import org.apache.lucene.analysis.payloads.IntegerEncoder;
+import org.apache.lucene.analysis.payloads.PayloadEncoder;
+import org.apache.lucene.queries.payloads.SpanPayloadCheckQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.spans.SpanQuery;
+import org.apache.lucene.util.BytesRef;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PayloadCheckQParserPlugin extends QParserPlugin {
+  public static final String NAME = "payload_check";
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @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 p = localParams.get("payloads");
+
+        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");
+        }
+
+        if (p == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'payloads' not specified");
+        }
+
+        FieldType ft = req.getCore().getLatestSchema().getFieldType(field);
+        Analyzer analyzer = ft.getQueryAnalyzer();
+        SpanQuery query = PayloadUtils.createSpanQuery(field, value, analyzer);
+
+        if (query == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "SpanQuery is null");
+        }
+
+        PayloadEncoder encoder = null;
+        String e = PayloadUtils.getPayloadEncoder(ft);
+        if ("float".equals(e)) {    // TODO: centralize this string->PayloadEncoder logic (see DelimitedPayloadTokenFilterFactory)
+          encoder = new FloatEncoder();
+        } else if ("integer".equals(e)) {
+          encoder = new IntegerEncoder();
+        } else if ("identity".equals(e)) {
+          encoder = new IdentityEncoder();
+        }
+
+        if (encoder == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "invalid encoder: " + e + " for field: " + field);
+        }
+
+        List<BytesRef> payloads = new ArrayList<>();
+        String[] rawPayloads = p.split(" ");  // since payloads (most likely) came in whitespace delimited, just split
+        for (String rawPayload : rawPayloads) {
+          if (rawPayload.length() > 0)
+            payloads.add(encoder.encode(rawPayload.toCharArray()));
+        }
+
+        return new SpanPayloadCheckQuery(query, payloads);
+      }
+    };
+
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/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..f9dd896
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/PayloadScoreQParserPlugin.java
@@ -0,0 +1,78 @@
+/*
+ * 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.lucene.analysis.Analyzer;
+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.SpanQuery;
+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 = PayloadUtils.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);
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/QParserPlugin.java b/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
index 872c618..d8dc29f 100644
--- a/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
@@ -80,6 +80,8 @@ public abstract class QParserPlugin implements NamedListInitializedPlugin, SolrI
     map.put(IGainTermsQParserPlugin.NAME, IGainTermsQParserPlugin.class);
     map.put(TextLogisticRegressionQParserPlugin.NAME, TextLogisticRegressionQParserPlugin.class);
     map.put(SignificantTermsQParserPlugin.NAME, SignificantTermsQParserPlugin.class);
+    map.put(PayloadScoreQParserPlugin.NAME, PayloadScoreQParserPlugin.class);
+    map.put(PayloadCheckQParserPlugin.NAME, PayloadCheckQParserPlugin.class);
 
     standardPlugins = Collections.unmodifiableMap(map);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
index f268baa..6032e0d 100644
--- a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
+++ b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
@@ -34,6 +34,7 @@ import org.apache.lucene.queries.function.docvalues.BoolDocValues;
 import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.queries.function.docvalues.LongDocValues;
 import org.apache.lucene.queries.function.valuesource.*;
+import org.apache.lucene.queries.payloads.PayloadFunction;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
@@ -76,6 +77,8 @@ import org.apache.solr.search.function.distance.StringDistanceFunction;
 import org.apache.solr.search.function.distance.VectorDistanceFunction;
 import org.apache.solr.search.join.ChildFieldValueSourceParser;
 import org.apache.solr.util.DateMathParser;
+import org.apache.solr.util.PayloadDecoder;
+import org.apache.solr.util.PayloadUtils;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 import org.locationtech.spatial4j.distance.DistanceUtils;
 
@@ -706,6 +709,47 @@ public abstract class ValueSourceParser implements NamedListInitializedPlugin {
       }
     });
 
+    addParser("payload", new ValueSourceParser() {
+      @Override
+      public ValueSource parse(FunctionQParser fp) throws SyntaxError {
+        // payload(field,value[,default, ['min|max|average|first']])
+        //   defaults to "average" and 0.0 default value
+
+        TInfo tinfo = parseTerm(fp); // would have made this parser a new separate class and registered it, but this handy method is private :/
+
+        ValueSource defaultValueSource;
+        if (fp.hasMoreArguments()) {
+          defaultValueSource = fp.parseValueSource();
+        } else {
+          defaultValueSource = new ConstValueSource(0.0f);
+        }
+
+        PayloadFunction payloadFunction = null;
+        String func = "average";
+        if (fp.hasMoreArguments()) {
+          func = fp.parseArg();
+        }
+        payloadFunction = PayloadUtils.getPayloadFunction(func);
+
+        // Support func="first" by payloadFunction=null
+        if(payloadFunction == null && !"first".equals(func)) {
+          // not "first" (or average, min, or max)
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid payload function: " + func);
+        }
+
+        FieldType fieldType = fp.getReq().getCore().getLatestSchema().getFieldTypeNoEx(tinfo.field);
+        PayloadDecoder decoder = PayloadUtils.getPayloadDecoder(fieldType);
+        return new FloatPayloadValueSource(
+            tinfo.field,
+            tinfo.val,
+            tinfo.indexedField,
+            tinfo.indexedBytes.get(),
+            decoder,
+            payloadFunction,
+            defaultValueSource);
+      }
+    });
+
     addParser("true", new ValueSourceParser() {
       @Override
       public ValueSource parse(FunctionQParser fp) {
@@ -1348,7 +1392,7 @@ abstract class Double2Parser extends NamedParser {
       final FunctionValues aVals =  a.getValues(context, readerContext);
       final FunctionValues bVals =  b.getValues(context, readerContext);
       return new DoubleDocValues(this) {
-         @Override
+        @Override
         public double doubleVal(int doc) throws IOException {
           return func(doc, aVals, bVals);
         }
@@ -1419,7 +1463,7 @@ class BoolConstValueSource extends ConstNumberSource {
     return this.constant == other.constant;
   }
 
-    @Override
+  @Override
   public int getInt() {
     return constant ? 1 : 0;
   }
@@ -1491,4 +1535,4 @@ class TestValueSource extends ValueSource {
   public SortField getSortField(boolean reverse) {
     return super.getSortField(reverse);
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/core/src/java/org/apache/solr/search/similarities/PayloadScoringSimilarityWrapper.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/similarities/PayloadScoringSimilarityWrapper.java b/solr/core/src/java/org/apache/solr/search/similarities/PayloadScoringSimilarityWrapper.java
new file mode 100644
index 0000000..4a86c42
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/similarities/PayloadScoringSimilarityWrapper.java
@@ -0,0 +1,80 @@
+/*
+ * 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.similarities;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.CollectionStatistics;
+import org.apache.lucene.search.TermStatistics;
+import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.util.PayloadDecoder;
+
+/**
+ * The computation Lucene's PayloadScoreQuery uses is SimScorer#computePayloadFactor.
+ * This wrapper delegates to a main similarity except for this one method.
+ */
+public class PayloadScoringSimilarityWrapper extends Similarity {
+  private Similarity delegate;
+  private PayloadDecoder decoder;
+
+  public PayloadScoringSimilarityWrapper(Similarity delegate, PayloadDecoder decoder) {
+    this.delegate = delegate;
+    this.decoder = decoder;
+  }
+
+  @Override
+  public String toString() {
+    return "PayloadScoring(" + delegate.toString() + ", decoder=" + decoder.toString() + ")";
+  }
+
+  @Override
+  public long computeNorm(FieldInvertState state) {
+    return delegate.computeNorm(state);
+  }
+
+  @Override
+  public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+    return delegate.computeWeight(boost, collectionStats, termStats);
+  }
+
+  @Override
+  public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
+    final SimScorer simScorer = delegate.simScorer(weight,context);
+    SimScorer payloadSimScorer = new SimScorer() {
+      @Override
+      public float score(int doc, float freq) throws IOException {
+        return simScorer.score(doc,freq);
+      }
+
+      @Override
+      public float computeSlopFactor(int distance) {
+        return simScorer.computeSlopFactor(distance);
+      }
+
+      @Override
+      public float computePayloadFactor(int doc, int start, int end, BytesRef payload) {
+        return decoder.decode(simScorer, doc, start, end, payload);
+      }
+    };
+
+    return payloadSimScorer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/core/src/java/org/apache/solr/search/similarities/SchemaSimilarityFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/similarities/SchemaSimilarityFactory.java b/solr/core/src/java/org/apache/solr/search/similarities/SchemaSimilarityFactory.java
index a71de18..3c942d2 100644
--- a/solr/core/src/java/org/apache/solr/search/similarities/SchemaSimilarityFactory.java
+++ b/solr/core/src/java/org/apache/solr/search/similarities/SchemaSimilarityFactory.java
@@ -16,6 +16,8 @@
  */
 package org.apache.solr.search.similarities;
 
+import java.util.HashMap;
+
 import org.apache.lucene.search.similarities.ClassicSimilarity;
 import org.apache.lucene.search.similarities.BM25Similarity;
 import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
@@ -28,12 +30,14 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.SimilarityFactory;
+import org.apache.solr.util.PayloadDecoder;
+import org.apache.solr.util.PayloadUtils;
 import org.apache.solr.util.plugin.SolrCoreAware;
 
 /**
  * <p>
  * <code>SimilarityFactory</code> that returns a global {@link PerFieldSimilarityWrapper}
- * that delegates to the field type, if it's configured.  For field type's that
+ * that delegates to the field type, if it's configured.  For field types that
  * do not have a <code>Similarity</code> explicitly configured, the global <code>Similarity</code> 
  * will use per fieldtype defaults -- either based on an explicitly configured 
  * <code>defaultSimFromFieldType</code> a sensible default depending on the {@link Version} 
@@ -45,7 +49,7 @@ import org.apache.solr.util.plugin.SolrCoreAware;
  * </ul>
  * <p>
  * The <code>defaultSimFromFieldType</code> option accepts the name of any fieldtype, and uses 
- * whatever <code>Similarity</code> is explicitly configured for that fieldType as thedefault for 
+ * whatever <code>Similarity</code> is explicitly configured for that fieldType as the default for
  * all other field types.  For example:
  * </p>
  * <pre class="prettyprint">
@@ -136,6 +140,7 @@ public class SchemaSimilarityFactory extends SimilarityFactory implements SolrCo
   
   private class SchemaSimilarity extends PerFieldSimilarityWrapper {
     private Similarity defaultSimilarity;
+    private HashMap<FieldType,PayloadDecoder> decoders;  // cache to avoid scanning token filters repeatedly, unnecessarily
 
     public SchemaSimilarity(Similarity defaultSimilarity) {
       this.defaultSimilarity = defaultSimilarity;
@@ -148,7 +153,19 @@ public class SchemaSimilarityFactory extends SimilarityFactory implements SolrCo
         return defaultSimilarity;
       } else {
         Similarity similarity = fieldType.getSimilarity();
-        return similarity == null ? defaultSimilarity : similarity;
+        similarity = similarity == null ? defaultSimilarity : similarity;
+
+        // Payload score handling: if field type has index-time payload encoding, wrap and computePayloadFactor accordingly
+        if (decoders == null) decoders = new HashMap<>();
+        PayloadDecoder decoder;
+        if (!decoders.containsKey(fieldType)) {
+          decoders.put(fieldType, PayloadUtils.getPayloadDecoder(fieldType));
+        }
+        decoder = decoders.get(fieldType);
+
+        if (decoder != null) similarity = new PayloadScoringSimilarityWrapper(similarity, decoder);
+
+        return similarity;
       }
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/core/src/java/org/apache/solr/util/PayloadDecoder.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/PayloadDecoder.java b/solr/core/src/java/org/apache/solr/util/PayloadDecoder.java
new file mode 100644
index 0000000..f9495b1
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/PayloadDecoder.java
@@ -0,0 +1,28 @@
+/*
+ * 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.util;
+
+import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Mirrors SimScorer#computePayloadFactor's signature
+ */
+public interface PayloadDecoder {
+  float decode(Similarity.SimScorer simScorer, int doc, int start, int end, BytesRef payload);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/core/src/java/org/apache/solr/util/PayloadUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/PayloadUtils.java b/solr/core/src/java/org/apache/solr/util/PayloadUtils.java
new file mode 100644
index 0000000..7927594
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/PayloadUtils.java
@@ -0,0 +1,134 @@
+/*
+ * 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.util;
+
+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.payloads.DelimitedPayloadTokenFilterFactory;
+import org.apache.lucene.analysis.payloads.NumericPayloadTokenFilterFactory;
+import org.apache.lucene.analysis.payloads.PayloadHelper;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.queries.payloads.AveragePayloadFunction;
+import org.apache.lucene.queries.payloads.MaxPayloadFunction;
+import org.apache.lucene.queries.payloads.MinPayloadFunction;
+import org.apache.lucene.queries.payloads.PayloadFunction;
+import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.search.spans.SpanNearQuery;
+import org.apache.lucene.search.spans.SpanQuery;
+import org.apache.lucene.search.spans.SpanTermQuery;
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.analysis.TokenizerChain;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.schema.FieldType;
+
+public class PayloadUtils {
+  public static String getPayloadEncoder(FieldType fieldType) {
+    // TODO: support custom payload encoding fields too somehow - maybe someone has a custom component that encodes payloads as floats
+    String encoder = null;
+    Analyzer a = fieldType.getIndexAnalyzer();
+    if (a instanceof TokenizerChain) {
+      // examine the indexing analysis chain for DelimitedPayloadTokenFilterFactory or NumericPayloadTokenFilterFactory
+      TokenizerChain tc = (TokenizerChain)a;
+      TokenFilterFactory[] factories = tc.getTokenFilterFactories();
+      for (TokenFilterFactory factory : factories) {
+        if (factory instanceof DelimitedPayloadTokenFilterFactory) {
+          encoder = factory.getOriginalArgs().get(DelimitedPayloadTokenFilterFactory.ENCODER_ATTR);
+          break;
+        }
+
+        if (factory instanceof NumericPayloadTokenFilterFactory) {
+          // encodes using `PayloadHelper.encodeFloat(payload)`
+          encoder = "float";
+          break;
+        }
+      }
+    }
+
+    return encoder;
+  }
+
+  public static PayloadDecoder getPayloadDecoder(FieldType fieldType) {
+    PayloadDecoder decoder = Similarity.SimScorer::computePayloadFactor;  // default to SimScorer's
+
+    String encoder = getPayloadEncoder(fieldType);
+
+    if ("integer".equals(encoder)) {
+      decoder = (Similarity.SimScorer simScorer, int doc, int start, int end, BytesRef payload) -> PayloadHelper.decodeInt(payload.bytes, payload.offset);
+    }
+    if ("float".equals(encoder)) {
+      decoder = (Similarity.SimScorer simScorer, int doc, int start, int end, BytesRef payload) -> PayloadHelper.decodeFloat(payload.bytes, payload.offset);
+    }
+    // encoder could be "identity" at this point, in the case of DelimitedTokenFilterFactory encoder="identity"
+
+    // TODO: support pluggable payload decoders?
+
+    return decoder;
+  }
+
+  public static PayloadFunction getPayloadFunction(String func) {
+    PayloadFunction payloadFunction = null;
+    if ("min".equals(func)) {
+      payloadFunction = new MinPayloadFunction();
+    }
+    if ("max".equals(func)) {
+      payloadFunction = new MaxPayloadFunction();
+    }
+    if ("average".equals(func)) {
+      payloadFunction = new AveragePayloadFunction();
+    }
+
+    return payloadFunction;
+  }
+
+  public static 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/6c565c00/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 cccf79a..179f804 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema11.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
@@ -459,11 +459,33 @@ valued. -->
         unknown fields indexed and/or stored by default --> 
    <!--dynamicField name="*" type="ignored" /-->
    
-
- <!-- Field to use to determine and enforce document uniqueness. 
-      Unless this field is marked with required="false", it will be a required field
-   -->
- <uniqueKey>id</uniqueKey>
+  <!-- For testing payload function -->
+  <dynamicField name="*_dpf" type="delimited_payloads_float" indexed="true"  stored="true"/>
+  <dynamicField name="*_dpi" type="delimited_payloads_int" indexed="true"  stored="true"/>
+  <dynamicField name="*_dps" type="delimited_payloads_string" indexed="true"  stored="true"/>
+  <fieldType name="delimited_payloads_float" class="solr.TextField" positionIncrementGap="100">
+    <analyzer>
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.DelimitedPayloadTokenFilterFactory" encoder="float"/>
+    </analyzer>
+  </fieldType>
+  <fieldType name="delimited_payloads_int" class="solr.TextField" positionIncrementGap="100">
+    <analyzer>
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.DelimitedPayloadTokenFilterFactory" encoder="integer"/>
+    </analyzer>
+  </fieldType>
+  <fieldType name="delimited_payloads_string" class="solr.TextField" positionIncrementGap="100">
+    <analyzer>
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.DelimitedPayloadTokenFilterFactory" encoder="identity"/>
+    </analyzer>
+  </fieldType>
+
+  <!-- Field to use to determine and enforce document uniqueness.
+       Unless this field is marked with required="false", it will be a required field
+    -->
+  <uniqueKey>id</uniqueKey>
 
  <!-- field for the QueryParser to use when an explicit fieldname is absent -->
  <defaultSearchField>text</defaultSearchField>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/core/src/test-files/solr/collection1/conf/schema15.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema15.xml b/solr/core/src/test-files/solr/collection1/conf/schema15.xml
index 57c6bf1..9b4f0b1 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema15.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema15.xml
@@ -618,6 +618,14 @@
 
   <dynamicField name="random_*" type="random"/>
 
+  <dynamicField name="*_dpf" type="delimited_payloads_float" indexed="true"  stored="true"/>
+  <fieldType name="delimited_payloads_float" class="solr.TextField" positionIncrementGap="100">
+    <analyzer>
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.DelimitedPayloadTokenFilterFactory" encoder="float"/>
+    </analyzer>
+  </fieldType>
+
 
   <defaultSearchField>text</defaultSearchField>
   <uniqueKey>id</uniqueKey>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
index c44cc1e..7745817 100644
--- a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
+++ b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
@@ -20,6 +20,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
+import junit.framework.AssertionFailedError;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryUtils;
 import org.apache.solr.SolrTestCaseJ4;
@@ -1131,4 +1132,45 @@ public class QueryEqualityTest extends SolrTestCaseJ4 {
       req.close();
     }
   }
-}
+
+  public void testPayloadScoreQuery() throws Exception {
+    // I don't see a precedent to test query inequality in here, so doing a `try`
+    // There was a bug with PayloadScoreQuery's .equals() method that said two queries were equal with different includeSpanScore settings
+
+    try {
+      assertQueryEquals
+          ("payload_score"
+              , "{!payload_score f=foo_dpf v=query func=min includeSpanScore=false}"
+              , "{!payload_score f=foo_dpf v=query func=min includeSpanScore=true}"
+          );
+      fail("queries should not have been equal");
+    } catch(AssertionFailedError e) {
+      assertTrue("queries were not equal, as expected", true);
+    }
+  }
+
+  public void testPayloadCheckQuery() throws Exception {
+    try {
+      assertQueryEquals
+          ("payload_check"
+              , "{!payload_check f=foo_dpf payloads=2}one"
+              , "{!payload_check f=foo_dpf payloads=2}two"
+          );
+      fail("queries should not have been equal");
+    } catch(AssertionFailedError e) {
+      assertTrue("queries were not equal, as expected", true);
+    }
+  }
+
+  public void testPayloadFunction() throws Exception {
+    SolrQueryRequest req = req("myField","bar_f");
+
+    try {
+      assertFuncEquals(req,
+          "payload(foo_dpf,some_term)",
+          "payload(foo_dpf,some_term)");
+    } finally {
+      req.close();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/core/src/test/org/apache/solr/search/TestPayloadCheckQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestPayloadCheckQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestPayloadCheckQParserPlugin.java
new file mode 100644
index 0000000..14bd833
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/TestPayloadCheckQParserPlugin.java
@@ -0,0 +1,73 @@
+/*
+ * 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 TestPayloadCheckQParserPlugin extends SolrTestCaseJ4 {
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig.xml", "schema11.xml");
+    createIndex();
+  }
+
+  public static void createIndex() {
+    assertU(adoc("id","1", "vals_dpi","A|1 B|2 C|3"));
+    assertU(adoc("id","2", "vals_dpf","one|1.0 two|2.0 three|3.0"));
+    assertU(adoc("id","3", "vals_dps","the|ARTICLE cat|NOUN jumped|VERB"));
+    assertU(commit());
+  }
+
+  @Test
+  public void test() {
+    clearIndex();
+
+    String[] should_matches = new String[] {
+        "{!payload_check f=vals_dpi v=A payloads=1}",
+        "{!payload_check f=vals_dpi v=B payloads=2}",
+        "{!payload_check f=vals_dpi v=C payloads=3}",
+        "{!payload_check f=vals_dpi payloads='1 2'}A B",
+        // "{!payload_check f=vals_dpi payloads='1 2.0'}A B",  // ideally this should pass, but IntegerEncoder can't handle "2.0"
+        "{!payload_check f=vals_dpi payloads='1 2 3'}A B C",
+
+        "{!payload_check f=vals_dpf payloads='1 2'}one two",
+        "{!payload_check f=vals_dpf payloads='1 2.0'}one two", // shows that FloatEncoder can handle "1"
+
+        "{!payload_check f=vals_dps payloads='NOUN VERB'}cat jumped"
+    };
+
+    String[] should_not_matches = new String[] {
+        "{!payload_check f=vals_dpi v=A payloads=2}",
+        "{!payload_check f=vals_dpi payloads='1 2'}B C",
+        "{!payload_check f=vals_dpi payloads='1 2 3'}A B",
+        "{!payload_check f=vals_dpi payloads='1 2'}A B C",
+        "{!payload_check f=vals_dpf payloads='1 2.0'}two three",
+        "{!payload_check f=vals_dps payloads='VERB NOUN'}cat jumped"
+    };
+
+    for(String should_match : should_matches) {
+      assertQ(should_match, req("fl","*,score", "q", should_match), "//result[@numFound='1']");
+    }
+
+    for(String should_not_match : should_not_matches) {
+      assertQ(should_not_match, req("fl","*,score", "q", should_not_match), "//result[@numFound='0']");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/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..34017c1
--- /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_dpf","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_dpf v=B func=min}"), "//float[@name='score']='2.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf v=mult func=min}"), "//float[@name='score']='50.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf v=mult func=max}"), "//float[@name='score']='100.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf v=mult func=average}"), "//float[@name='score']='75.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=min}A B"), "//float[@name='score']='1.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=min}B C"), "//float[@name='score']='2.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=max}B C"), "//float[@name='score']='3.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=average}B C"), "//float[@name='score']='2.5'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf 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_dpf v=A func=min}"), "//float[@name='score']='1.0'");
+    assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf v=A func=min includeSpanScore=true}"), "//float[@name='score']='0.25811607'");
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java b/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
index 92f2863..f218bd9 100644
--- a/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
+++ b/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
@@ -462,6 +462,34 @@ public class TestFunctionQuery extends SolrTestCaseJ4 {
   }
 
   @Test
+  public void testPayloadFunction() {
+    clearIndex();
+
+    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", "default_f", "42.0"));
+    assertU(commit());
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,A)"), "//float[@name='score']='1.0'");
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,B)"), "//float[@name='score']='2.0'");
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,C,0)"), "//float[@name='score']='3.0'");
+
+    // Test defaults, constant, field, and function value sources
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,D,37.0)"), "//float[@name='score']='37.0'");
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,E,default_f)"), "//float[@name='score']='42.0'");
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,E,mul(2,default_f))"), "//float[@name='score']='84.0'");
+
+    // Test PayloadFunction's for multiple terms, average being the default
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,mult,0.0,min)"), "//float[@name='score']='50.0'");
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,mult,0.0,max)"), "//float[@name='score']='100.0'");
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,mult,0.0,average)"), "//float[@name='score']='75.0'");
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,mult)"), "//float[@name='score']='75.0'");
+
+    // Test special "first" function, by checking the other functions with same term too
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,x,0.0,min)"), "//float[@name='score']='19.0'");
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,x,0.0,max)"), "//float[@name='score']='37.0'");
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,x,0.0,average)"), "//float[@name='score']='26.0'");
+    assertQ(req("fl","*,score","q", "{!func}payload(vals_dp,x,0.0,first)"), "//float[@name='score']='22.0'");
+  }
+
+  @Test
   public void testSortByFunc() throws Exception {
     assertU(adoc("id",    "1",   "const_s", "xx", 
                  "x_i",   "100", "1_s", "a",
@@ -772,7 +800,7 @@ public class TestFunctionQuery extends SolrTestCaseJ4 {
     assertU(commit());
 
     // it's important that these functions not only use fields that
-    // out doc have no values for, but also that that no other doc ever added
+    // our doc has no values for, but also that no other doc ever added
     // to the index might have ever had a value for, so that the segment
     // term metadata doesn't exist
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c565c00/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema b/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema
index 68f6867..80a58fb 100644
--- a/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema
+++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema
@@ -177,6 +177,11 @@
     <dynamicField name="*_tdt" type="tdate"  indexed="true"  stored="true"/>
     <dynamicField name="*_tdts" type="tdates"  indexed="true"  stored="true"/>
 
+    <!-- payloaded dynamic fields -->
+    <dynamicField name="*_dpf" type="delimited_payloads_float" indexed="true"  stored="true"/>
+    <dynamicField name="*_dpi" type="delimited_payloads_int" indexed="true"  stored="true"/>
+    <dynamicField name="*_dps" type="delimited_payloads_string" indexed="true"  stored="true"/>
+
     <dynamicField name="*_c"   type="currency" indexed="true"  stored="true"/>
 
     <dynamicField name="ignored_*" type="ignored" multiValued="true"/>
@@ -600,6 +605,26 @@
     <fieldType name="location_rpt" class="solr.SpatialRecursivePrefixTreeFieldType"
                geo="true" distErrPct="0.025" maxDistErr="0.001" distanceUnits="kilometers" />
 
+    <!-- Payloaded field types -->
+    <fieldType name="delimited_payloads_float" stored="false" indexed="true" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+        <filter class="solr.DelimitedPayloadTokenFilterFactory" encoder="float"/>
+      </analyzer>
+    </fieldType>
+    <fieldType name="delimited_payloads_int" stored="false" indexed="true" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+        <filter class="solr.DelimitedPayloadTokenFilterFactory" encoder="integer"/>
+      </analyzer>
+    </fieldType>
+    <fieldType name="delimited_payloads_string" stored="false" indexed="true" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+        <filter class="solr.DelimitedPayloadTokenFilterFactory" encoder="identity"/>
+      </analyzer>
+    </fieldType>
+
     <!-- Money/currency field type. See http://wiki.apache.org/solr/MoneyFieldType
         Parameters:
           defaultCurrency: Specifies the default currency if none specified. Defaults to "USD"