You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cp...@apache.org on 2016/12/08 18:44:02 UTC

[03/13] lucene-solr:branch_6x: SOLR-8542: Adds Solr Learning to Rank (LTR) plugin for reranking results with machine learning models. (Michael Nilsson, Diego Ceccarelli, Joshua Pantony, Jon Dorando, Naveen Santhapuri, Alessandro Benedetti, David Grohmann

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java
new file mode 100644
index 0000000..68961d2
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java
@@ -0,0 +1,251 @@
+/*
+ * 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.ltr;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.ReaderUtil;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.store.Directory;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.feature.Feature;
+import org.apache.solr.ltr.feature.ValueFeature;
+import org.apache.solr.ltr.model.LTRScoringModel;
+import org.apache.solr.ltr.model.ModelException;
+import org.apache.solr.ltr.model.TestLinearModel;
+import org.apache.solr.ltr.norm.IdentityNormalizer;
+import org.apache.solr.ltr.norm.Normalizer;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestSelectiveWeightCreation extends TestRerankBase {
+  private IndexSearcher getSearcher(IndexReader r) {
+    final IndexSearcher searcher = newSearcher(r, false, false);
+    return searcher;
+  }
+
+  private static List<Feature> makeFeatures(int[] featureIds) {
+    final List<Feature> features = new ArrayList<>();
+    for (final int i : featureIds) {
+      Map<String,Object> params = new HashMap<String,Object>();
+      params.put("value", i);
+      final Feature f = Feature.getInstance(solrResourceLoader,
+          ValueFeature.class.getCanonicalName(),
+          "f" + i, params);
+      f.setIndex(i);
+      features.add(f);
+    }
+    return features;
+  }
+
+  private static Map<String,Object> makeFeatureWeights(List<Feature> features) {
+    final Map<String,Object> nameParams = new HashMap<String,Object>();
+    final HashMap<String,Double> modelWeights = new HashMap<String,Double>();
+    for (final Feature feat : features) {
+      modelWeights.put(feat.getName(), 0.1);
+    }
+    nameParams.put("weights", modelWeights);
+    return nameParams;
+  }
+
+  private LTRScoringQuery.ModelWeight performQuery(TopDocs hits,
+      IndexSearcher searcher, int docid, LTRScoringQuery model) throws IOException,
+      ModelException {
+    final List<LeafReaderContext> leafContexts = searcher.getTopReaderContext()
+        .leaves();
+    final int n = ReaderUtil.subIndex(hits.scoreDocs[0].doc, leafContexts);
+    final LeafReaderContext context = leafContexts.get(n);
+    final int deBasedDoc = hits.scoreDocs[0].doc - context.docBase;
+
+    final Weight weight = searcher.createNormalizedWeight(model, true);
+    final Scorer scorer = weight.scorer(context);
+
+    // rerank using the field final-score
+    scorer.iterator().advance(deBasedDoc);
+    scorer.score();
+    assertTrue(weight instanceof LTRScoringQuery.ModelWeight);
+    final LTRScoringQuery.ModelWeight modelWeight = (LTRScoringQuery.ModelWeight) weight;
+    return modelWeight;
+
+  }
+
+
+  @BeforeClass
+  public static void before() throws Exception {
+    setuptest("solrconfig-ltr.xml", "schema.xml");
+
+    assertU(adoc("id", "1", "title", "w1 w3", "description", "w1", "popularity",
+        "1"));
+    assertU(adoc("id", "2", "title", "w2", "description", "w2", "popularity",
+        "2"));
+    assertU(adoc("id", "3", "title", "w3", "description", "w3", "popularity",
+        "3"));
+    assertU(adoc("id", "4", "title", "w4 w3", "description", "w4", "popularity",
+        "4"));
+    assertU(adoc("id", "5", "title", "w5", "description", "w5", "popularity",
+        "5"));
+    assertU(commit());
+
+    loadFeatures("external_features.json");
+    loadModels("external_model.json");
+    loadModels("external_model_store.json");
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    aftertest();
+  }
+
+  @Test
+  public void testScoringQueryWeightCreation() throws IOException, ModelException {
+    final Directory dir = newDirectory();
+    final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+    Document doc = new Document();
+    doc.add(newStringField("id", "0", Field.Store.YES));
+    doc.add(newTextField("field", "wizard the the the the the oz",
+        Field.Store.NO));
+    doc.add(new FloatDocValuesField("final-score", 1.0f));
+
+    w.addDocument(doc);
+    doc = new Document();
+    doc.add(newStringField("id", "1", Field.Store.YES));
+    // 1 extra token, but wizard and oz are close;
+    doc.add(newTextField("field", "wizard oz the the the the the the",
+        Field.Store.NO));
+    doc.add(new FloatDocValuesField("final-score", 2.0f));
+    w.addDocument(doc);
+
+    final IndexReader r = w.getReader();
+    w.close();
+
+    // Do ordinary BooleanQuery:
+    final BooleanQuery.Builder bqBuilder = new BooleanQuery.Builder();
+    bqBuilder.add(new TermQuery(new Term("field", "wizard")), BooleanClause.Occur.SHOULD);
+    bqBuilder.add(new TermQuery(new Term("field", "oz")), BooleanClause.Occur.SHOULD);
+    final IndexSearcher searcher = getSearcher(r);
+    // first run the standard query
+    final TopDocs hits = searcher.search(bqBuilder.build(), 10);
+    assertEquals(2, hits.totalHits);
+    assertEquals("0", searcher.doc(hits.scoreDocs[0].doc).get("id"));
+    assertEquals("1", searcher.doc(hits.scoreDocs[1].doc).get("id"));
+
+    List<Feature> features = makeFeatures(new int[] {0, 1, 2});
+    final List<Feature> allFeatures = makeFeatures(new int[] {0, 1, 2, 3, 4, 5,
+        6, 7, 8, 9});
+    final List<Normalizer> norms = new ArrayList<>();
+    for (int k=0; k < features.size(); ++k){
+        norms.add(IdentityNormalizer.INSTANCE);
+    }
+
+    // when features are NOT requested in the response, only the modelFeature weights should be created
+    final LTRScoringModel ltrScoringModel1 = TestLinearModel.createLinearModel("test",
+        features, norms, "test", allFeatures,
+        makeFeatureWeights(features));
+    LTRScoringQuery.ModelWeight modelWeight = performQuery(hits, searcher,
+        hits.scoreDocs[0].doc, new LTRScoringQuery(ltrScoringModel1, false)); // features not requested in response
+    LTRScoringQuery.FeatureInfo[] featuresInfo = modelWeight.getFeaturesInfo();
+
+    assertEquals(features.size(), modelWeight.getModelFeatureValuesNormalized().length);
+    int validFeatures = 0;
+    for (int i=0; i < featuresInfo.length; ++i){
+      if (featuresInfo[i] != null && featuresInfo[i].isUsed()){
+        validFeatures += 1;
+      }
+    }
+    assertEquals(validFeatures, features.size());
+
+    // when features are requested in the response, weights should be created for all features
+    final LTRScoringModel ltrScoringModel2 = TestLinearModel.createLinearModel("test",
+        features, norms, "test", allFeatures,
+        makeFeatureWeights(features));
+    modelWeight = performQuery(hits, searcher,
+        hits.scoreDocs[0].doc, new LTRScoringQuery(ltrScoringModel2, true)); // features requested in response
+    featuresInfo = modelWeight.getFeaturesInfo();
+
+    assertEquals(features.size(), modelWeight.getModelFeatureValuesNormalized().length);
+    assertEquals(allFeatures.size(), modelWeight.getExtractedFeatureWeights().length);
+
+    validFeatures = 0;
+    for (int i=0; i < featuresInfo.length; ++i){
+      if (featuresInfo[i] != null && featuresInfo[i].isUsed()){
+        validFeatures += 1;
+      }
+    }
+    assertEquals(validFeatures, allFeatures.size());
+
+    assertU(delI("0"));assertU(delI("1"));
+    r.close();
+    dir.close();
+  }
+
+
+  @Test
+  public void testSelectiveWeightsRequestFeaturesFromDifferentStore() throws Exception {
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("fl", "*,score");
+    query.add("rows", "4");
+    query.add("rq", "{!ltr reRankDocs=4 model=externalmodel efi.user_query=w3}");
+    query.add("fl", "fv:[fv]");
+
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='3'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='4'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv=='matchedTitle:1.0;titlePhraseMatch:0.40254828'"); // extract all features in default store
+
+    query.remove("fl");
+    query.remove("rq");
+    query.add("fl", "*,score");
+    query.add("rq", "{!ltr reRankDocs=4 model=externalmodel efi.user_query=w3}");
+    query.add("fl", "fv:[fv store=fstore4 efi.myPop=3]");
+
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==0.999");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv=='popularity:3.0;originalScore:1.0'"); // extract all features from fstore4
+
+
+    query.remove("fl");
+    query.remove("rq");
+    query.add("fl", "*,score");
+    query.add("rq", "{!ltr reRankDocs=4 model=externalmodelstore efi.user_query=w3 efi.myconf=0.8}");
+    query.add("fl", "fv:[fv store=fstore4 efi.myPop=3]");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'"); // score using fstore2 used by externalmodelstore
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==0.7992");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv=='popularity:3.0;originalScore:1.0'"); // extract all features from fstore4
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestEdisMaxSolrFeature.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestEdisMaxSolrFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestEdisMaxSolrFeature.java
new file mode 100644
index 0000000..cd63b5c
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestEdisMaxSolrFeature.java
@@ -0,0 +1,76 @@
+/*
+ * 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.ltr.feature;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.TestRerankBase;
+import org.apache.solr.ltr.model.LinearModel;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestEdisMaxSolrFeature extends TestRerankBase {
+
+  @BeforeClass
+  public static void before() throws Exception {
+    setuptest("solrconfig-ltr.xml", "schema.xml");
+
+    assertU(adoc("id", "1", "title", "w1", "description", "w1", "popularity",
+        "1"));
+    assertU(adoc("id", "2", "title", "w2 2asd asdd didid", "description",
+        "w2 2asd asdd didid", "popularity", "2"));
+    assertU(adoc("id", "3", "title", "w3", "description", "w3", "popularity",
+        "3"));
+    assertU(adoc("id", "4", "title", "w4", "description", "w4", "popularity",
+        "4"));
+    assertU(adoc("id", "5", "title", "w5", "description", "w5", "popularity",
+        "5"));
+    assertU(adoc("id", "6", "title", "w1 w2", "description", "w1 w2",
+        "popularity", "6"));
+    assertU(adoc("id", "7", "title", "w1 w2 w3 w4 w5", "description",
+        "w1 w2 w3 w4 w5 w8", "popularity", "7"));
+    assertU(adoc("id", "8", "title", "w1 w1 w1 w2 w2 w8", "description",
+        "w1 w1 w1 w2 w2", "popularity", "8"));
+    assertU(commit());
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    aftertest();
+  }
+
+  @Test
+  public void testEdisMaxSolrFeature() throws Exception {
+    loadFeature(
+        "SomeEdisMax",
+        SolrFeature.class.getCanonicalName(),
+        "{\"q\":\"{!edismax qf='title description' pf='description' mm=100% boost='pow(popularity, 0.1)' v='w1' tie=0.1}\"}");
+
+    loadModel("EdisMax-model", LinearModel.class.getCanonicalName(),
+        new String[] {"SomeEdisMax"}, "{\"weights\":{\"SomeEdisMax\":1.0}}");
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("title:w1");
+    query.add("fl", "*, score");
+    query.add("rows", "4");
+
+    query.add("rq", "{!ltr model=EdisMax-model reRankDocs=4}");
+    query.set("debugQuery", "on");
+    restTestHarness.query("/query" + query.toQueryString());
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==4");
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java
new file mode 100644
index 0000000..8c00758
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java
@@ -0,0 +1,157 @@
+/*
+ * 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.ltr.feature;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.TestRerankBase;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestExternalFeatures extends TestRerankBase {
+
+  @BeforeClass
+  public static void before() throws Exception {
+    setuptest("solrconfig-ltr.xml", "schema.xml");
+
+    assertU(adoc("id", "1", "title", "w1", "description", "w1", "popularity",
+        "1"));
+    assertU(adoc("id", "2", "title", "w2", "description", "w2", "popularity",
+        "2"));
+    assertU(adoc("id", "3", "title", "w3", "description", "w3", "popularity",
+        "3"));
+    assertU(adoc("id", "4", "title", "w4", "description", "w4", "popularity",
+        "4"));
+    assertU(adoc("id", "5", "title", "w5", "description", "w5", "popularity",
+        "5"));
+    assertU(commit());
+
+    loadFeatures("external_features.json");
+    loadModels("external_model.json");
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    aftertest();
+  }
+
+  @Test
+  public void testEfiInTransformerShouldNotChangeOrderOfRerankedResults() throws Exception {
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("fl", "*,score");
+    query.add("rows", "3");
+
+    // Regular scores
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==1.0");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='2'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/score==1.0");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='3'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/score==1.0");
+
+    query.add("fl", "[fv]");
+    query.add("rq", "{!ltr reRankDocs=3 model=externalmodel efi.user_query=w3}");
+
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='3'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==0.999");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='1'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/score==0.0");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='2'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/score==0.0");
+
+    // Adding an efi in the transformer should not affect the rq ranking with a
+    // different value for efi of the same parameter
+    query.remove("fl");
+    query.add("fl", "id,[fv efi.user_query=w2]");
+
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='3'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='1'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='2'");
+  }
+
+  @Test
+  public void testFeaturesUseStopwordQueryReturnEmptyFeatureVector() throws Exception {
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("fl", "*,score,fv:[fv]");
+    query.add("rows", "1");
+    // Stopword only query passed in
+    query.add("rq", "{!ltr reRankDocs=3 model=externalmodel efi.user_query='a'}");
+
+    // Features are query title matches, which remove stopwords, leaving blank query, so no matches
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv==''");
+  }
+
+  @Test
+  public void testEfiFeatureExtraction() throws Exception {
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("rows", "1");
+
+    // Features we're extracting depend on external feature info not passed in
+    query.add("fl", "[fv]");
+    assertJQ("/query" + query.toQueryString(), "/error/msg=='Exception from createWeight for SolrFeature [name=matchedTitle, params={q={!terms f=title}${user_query}}] SolrFeatureWeight requires efi parameter that was not passed in request.'");
+
+    // Adding efi in features section should make it work
+    query.remove("fl");
+    query.add("fl", "score,fvalias:[fv store=fstore2 efi.myconf=2.3]");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fvalias=='confidence:2.3;originalScore:1.0'");
+
+    // Adding efi in transformer + rq should still use the transformer's params for feature extraction
+    query.remove("fl");
+    query.add("fl", "score,fvalias:[fv store=fstore2 efi.myconf=2.3]");
+    query.add("rq", "{!ltr reRankDocs=3 model=externalmodel efi.user_query=w3}");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fvalias=='confidence:2.3;originalScore:1.0'");
+  }
+
+  @Test
+  public void featureExtraction_valueFeatureImplicitlyNotRequired_shouldNotScoreFeature() throws Exception {
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("rows", "1");
+
+    // Efi is explicitly not required, so we do not score the feature
+    query.remove("fl");
+    query.add("fl", "fvalias:[fv store=fstore2]");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fvalias=='originalScore:0.0'");
+  }
+
+  @Test
+  public void featureExtraction_valueFeatureExplicitlyNotRequired_shouldNotScoreFeature() throws Exception {
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("rows", "1");
+
+    // Efi is explicitly not required, so we do not score the feature
+    query.remove("fl");
+    query.add("fl", "fvalias:[fv store=fstore3]");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fvalias=='originalScore:0.0'");
+  }
+
+  @Test
+  public void featureExtraction_valueFeatureRequired_shouldThrowException() throws Exception {
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("rows", "1");
+
+    // Using nondefault store should still result in error with no efi when it is required (myPop)
+    query.remove("fl");
+    query.add("fl", "fvalias:[fv store=fstore4]");
+    assertJQ("/query" + query.toQueryString(), "/error/msg=='Exception from createWeight for ValueFeature [name=popularity, params={value=${myPop}, required=true}] ValueFeatureWeight requires efi parameter that was not passed in request.'");
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalValueFeatures.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalValueFeatures.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalValueFeatures.java
new file mode 100644
index 0000000..bc073cb
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalValueFeatures.java
@@ -0,0 +1,86 @@
+/*
+ * 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.ltr.feature;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.TestRerankBase;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestExternalValueFeatures extends TestRerankBase {
+
+  @BeforeClass
+  public static void before() throws Exception {
+    setuptest("solrconfig-ltr.xml", "schema.xml");
+
+    assertU(adoc("id", "1", "title", "w1", "description", "w1", "popularity",
+        "1"));
+    assertU(adoc("id", "2", "title", "w2", "description", "w2", "popularity",
+        "2"));
+    assertU(adoc("id", "3", "title", "w3", "description", "w3", "popularity",
+        "3"));
+    assertU(adoc("id", "4", "title", "w4", "description", "w4", "popularity",
+        "4"));
+    assertU(adoc("id", "5", "title", "w5", "description", "w5", "popularity",
+        "5"));
+    assertU(commit());
+
+    loadFeatures("external_features_for_sparse_processing.json");
+    loadModels("multipleadditivetreesmodel_external_binary_features.json");
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    aftertest();
+  }
+
+  @Test
+  public void efiFeatureProcessing_oneEfiMissing_shouldNotCalculateMissingFeature() throws Exception {
+    SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("fl", "*,score,features:[fv]");
+    query.add("rows", "3");
+    query.add("fl", "[fv]");
+    query.add("rq", "{!ltr reRankDocs=3 model=external_model_binary_feature efi.user_device_tablet=1}");
+
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/features=='user_device_tablet:1.0'");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/score==65.0");
+  }
+
+  @Test
+  public void efiFeatureProcessing_allEfisMissing_shouldReturnZeroScore() throws Exception {
+    SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("fl", "*,score,features:[fv]");
+    query.add("rows", "3");
+
+    query.add("fl", "[fv]");
+    query
+        .add("rq", "{!ltr reRankDocs=3 model=external_model_binary_feature}");
+
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/features==''");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/score==0.0");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureExtractionFromMultipleSegments.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureExtractionFromMultipleSegments.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureExtractionFromMultipleSegments.java
new file mode 100644
index 0000000..7658f62
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureExtractionFromMultipleSegments.java
@@ -0,0 +1,105 @@
+/*
+ * 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.ltr.feature;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.TestRerankBase;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.noggit.ObjectBuilder;
+
+
+public class TestFeatureExtractionFromMultipleSegments extends TestRerankBase {
+  static final String AB = "abcdefghijklmnopqrstuvwxyz";
+
+  static String randomString( int len ){
+    StringBuilder sb = new StringBuilder( len );
+    for( int i = 0; i < len; i++ ) {
+      sb.append( AB.charAt( random().nextInt(AB.length()) ) );
+    }
+    return sb.toString();
+ }
+
+  @BeforeClass
+  public static void before() throws Exception {
+    // solrconfig-multiseg.xml contains the merge policy to restrict merging
+    setuptest("solrconfig-multiseg.xml", "schema.xml");
+    // index 400 documents
+    for(int i = 0; i<400;i=i+20) {
+      assertU(adoc("id", new Integer(i).toString(),   "popularity", "201", "description", "apple is a company " + randomString(i%6+3), "normHits", "0.1"));
+      assertU(adoc("id", new Integer(i+1).toString(), "popularity", "201", "description", "d " + randomString(i%6+3), "normHits", "0.11"));
+
+      assertU(adoc("id", new Integer(i+2).toString(), "popularity", "201", "description", "apple is a company too " + randomString(i%6+3), "normHits", "0.1"));
+      assertU(adoc("id", new Integer(i+3).toString(), "popularity", "201", "description", "new york city is big apple " + randomString(i%6+3), "normHits", "0.11"));
+
+      assertU(adoc("id", new Integer(i+6).toString(), "popularity", "301", "description", "function name " + randomString(i%6+3), "normHits", "0.1"));
+      assertU(adoc("id", new Integer(i+7).toString(), "popularity", "301", "description", "function " + randomString(i%6+3), "normHits", "0.1"));
+
+      assertU(adoc("id", new Integer(i+8).toString(), "popularity", "301", "description", "This is a sample function for testing " + randomString(i%6+3), "normHits", "0.1"));
+      assertU(adoc("id", new Integer(i+9).toString(), "popularity", "301", "description", "Function to check out stock prices "+randomString(i%6+3), "normHits", "0.1"));
+      assertU(adoc("id", new Integer(i+10).toString(),"popularity", "301", "description", "Some descriptions "+randomString(i%6+3), "normHits", "0.1"));
+
+      assertU(adoc("id", new Integer(i+11).toString(), "popularity", "201", "description", "apple apple is a company " + randomString(i%6+3), "normHits", "0.1"));
+      assertU(adoc("id", new Integer(i+12).toString(), "popularity", "201", "description", "Big Apple is New York.", "normHits", "0.01"));
+      assertU(adoc("id", new Integer(i+13).toString(), "popularity", "201", "description", "New some York is Big. "+ randomString(i%6+3), "normHits", "0.1"));
+
+      assertU(adoc("id", new Integer(i+14).toString(), "popularity", "201", "description", "apple apple is a company " + randomString(i%6+3), "normHits", "0.1"));
+      assertU(adoc("id", new Integer(i+15).toString(), "popularity", "201", "description", "Big Apple is New York.", "normHits", "0.01"));
+      assertU(adoc("id", new Integer(i+16).toString(), "popularity", "401", "description", "barack h", "normHits", "0.0"));
+      assertU(adoc("id", new Integer(i+17).toString(), "popularity", "201", "description", "red delicious apple " + randomString(i%6+3), "normHits", "0.1"));
+      assertU(adoc("id", new Integer(i+18).toString(), "popularity", "201", "description", "nyc " + randomString(i%6+3), "normHits", "0.11"));
+    }
+
+    assertU(commit());
+
+    loadFeatures("comp_features.json");
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    aftertest();
+  }
+
+  @Test
+  public void testFeatureExtractionFromMultipleSegments() throws Exception {
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("{!edismax qf='description^1' boost='sum(product(pow(normHits, 0.7), 1600), .1)' v='apple'}");
+    // request 100 rows, if any rows are fetched from the second or subsequent segments the tests should succeed if LTRRescorer::extractFeaturesInfo() advances the doc iterator properly
+    int numRows = 100;
+    query.add("rows", (new Integer(numRows)).toString());
+    query.add("wt", "json");
+    query.add("fq", "popularity:201");
+    query.add("fl", "*, score,id,normHits,description,fv:[features store='feature-store-6' format='dense' efi.user_text='apple']");
+    String res = restTestHarness.query("/query" + query.toQueryString());
+
+    Map<String,Object> resultJson = (Map<String,Object>) ObjectBuilder.fromJSON(res);
+
+    List<Map<String,Object>> docs = (List<Map<String,Object>>)((Map<String,Object>)resultJson.get("response")).get("docs");
+    int passCount = 0;
+    for (final Map<String,Object> doc : docs) {
+       String features = (String)doc.get("fv");
+       assert(features.length() > 0);
+       ++passCount;
+    }
+    assert(passCount == numRows);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLogging.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLogging.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLogging.java
new file mode 100644
index 0000000..14e2903
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLogging.java
@@ -0,0 +1,254 @@
+/*
+ * 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.ltr.feature;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.TestRerankBase;
+import org.apache.solr.ltr.model.LinearModel;
+import org.apache.solr.ltr.store.FeatureStore;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestFeatureLogging extends TestRerankBase {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    setuptest();
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    aftertest();
+  }
+
+  @Test
+  public void testGeneratedFeatures() throws Exception {
+    loadFeature("c1", ValueFeature.class.getCanonicalName(), "test1",
+        "{\"value\":1.0}");
+    loadFeature("c2", ValueFeature.class.getCanonicalName(), "test1",
+        "{\"value\":2.0}");
+    loadFeature("c3", ValueFeature.class.getCanonicalName(), "test1",
+        "{\"value\":3.0}");
+    loadFeature("pop", FieldValueFeature.class.getCanonicalName(), "test1",
+        "{\"field\":\"popularity\"}");
+    loadFeature("nomatch", SolrFeature.class.getCanonicalName(), "test1",
+        "{\"q\":\"{!terms f=title}foobarbat\"}");
+    loadFeature("yesmatch", SolrFeature.class.getCanonicalName(), "test1",
+        "{\"q\":\"{!terms f=popularity}2\"}");
+
+    loadModel("sum1", LinearModel.class.getCanonicalName(), new String[] {
+        "c1", "c2", "c3"}, "test1",
+        "{\"weights\":{\"c1\":1.0,\"c2\":1.0,\"c3\":1.0}}");
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("title:bloomberg");
+    query.add("fl", "title,description,id,popularity,[fv]");
+    query.add("rows", "3");
+    query.add("debugQuery", "on");
+    query.add("rq", "{!ltr reRankDocs=3 model=sum1}");
+
+    restTestHarness.query("/query" + query.toQueryString());
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[0]/=={'title':'bloomberg bloomberg ', 'description':'bloomberg','id':'7', 'popularity':2,  '[fv]':'c1:1.0;c2:2.0;c3:3.0;pop:2.0;yesmatch:1.0'}");
+
+    query.remove("fl");
+    query.add("fl", "[fv]");
+    query.add("rows", "3");
+    query.add("rq", "{!ltr reRankDocs=3 model=sum1}");
+
+    restTestHarness.query("/query" + query.toQueryString());
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/=={'[fv]':'c1:1.0;c2:2.0;c3:3.0;pop:2.0;yesmatch:1.0'}");
+    query.remove("rq");
+
+    // set logging at false but still asking for feature, and it should work anyway
+    query.add("rq", "{!ltr reRankDocs=3 model=sum1}");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/=={'[fv]':'c1:1.0;c2:2.0;c3:3.0;pop:2.0;yesmatch:1.0'}");
+
+
+  }
+
+  @Test
+  public void testDefaultStoreFeatureExtraction() throws Exception {
+    loadFeature("defaultf1", ValueFeature.class.getCanonicalName(),
+        FeatureStore.DEFAULT_FEATURE_STORE_NAME,
+        "{\"value\":1.0}");
+    loadFeature("store8f1", ValueFeature.class.getCanonicalName(),
+        "store8",
+        "{\"value\":2.0}");
+    loadFeature("store9f1", ValueFeature.class.getCanonicalName(),
+        "store9",
+        "{\"value\":3.0}");
+    loadModel("store9m1", LinearModel.class.getCanonicalName(),
+      new String[] {"store9f1"},
+      "store9",
+      "{\"weights\":{\"store9f1\":1.0}}");
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("id:7");
+    query.add("rows", "1");
+
+    // No store specified, use default store for extraction
+    query.add("fl", "fv:[fv]");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/=={'fv':'defaultf1:1.0'}");
+
+    // Store specified, use store for extraction
+    query.remove("fl");
+    query.add("fl", "fv:[fv store=store8]");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/=={'fv':'store8f1:2.0'}");
+
+    // Store specified + model specified, use store for extraction
+    query.add("rq", "{!ltr reRankDocs=3 model=store9m1}");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/=={'fv':'store8f1:2.0'}");
+
+    // No store specified + model specified, use model store for extraction
+    query.remove("fl");
+    query.add("fl", "fv:[fv]");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/=={'fv':'store9f1:3.0'}");
+  }
+
+
+  @Test
+  public void testGeneratedGroup() throws Exception {
+    loadFeature("c1", ValueFeature.class.getCanonicalName(), "testgroup",
+        "{\"value\":1.0}");
+    loadFeature("c2", ValueFeature.class.getCanonicalName(), "testgroup",
+        "{\"value\":2.0}");
+    loadFeature("c3", ValueFeature.class.getCanonicalName(), "testgroup",
+        "{\"value\":3.0}");
+    loadFeature("pop", FieldValueFeature.class.getCanonicalName(), "testgroup",
+        "{\"field\":\"popularity\"}");
+
+    loadModel("sumgroup", LinearModel.class.getCanonicalName(), new String[] {
+        "c1", "c2", "c3"}, "testgroup",
+        "{\"weights\":{\"c1\":1.0,\"c2\":1.0,\"c3\":1.0}}");
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("title:bloomberg");
+    query.add("fl", "*,[fv]");
+    query.add("debugQuery", "on");
+
+    query.remove("fl");
+    query.add("fl", "fv:[fv]");
+    query.add("rows", "3");
+    query.add("group", "true");
+    query.add("group.field", "title");
+
+    query.add("rq", "{!ltr reRankDocs=3 model=sumgroup}");
+
+    restTestHarness.query("/query" + query.toQueryString());
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/grouped/title/groups/[0]/doclist/docs/[0]/=={'fv':'c1:1.0;c2:2.0;c3:3.0;pop:5.0'}");
+
+    query.remove("fl");
+    query.add("fl", "fv:[fv fvwt=json]");
+    restTestHarness.query("/query" + query.toQueryString());
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/grouped/title/groups/[0]/doclist/docs/[0]/fv/=={'c1':1.0,'c2':2.0,'c3':3.0,'pop':5.0}");
+    query.remove("fl");
+    query.add("fl", "fv:[fv fvwt=json]");
+
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/grouped/title/groups/[0]/doclist/docs/[0]/fv/=={'c1':1.0,'c2':2.0,'c3':3.0,'pop':5.0}");
+  }
+
+  @Test
+  public void testSparseDenseFeatures() throws Exception {
+    loadFeature("match", SolrFeature.class.getCanonicalName(), "test4",
+        "{\"q\":\"{!terms f=title}different\"}");
+    loadFeature("c4", ValueFeature.class.getCanonicalName(), "test4",
+        "{\"value\":1.0}");
+
+    loadModel("sum4", LinearModel.class.getCanonicalName(), new String[] {
+        "match"}, "test4",
+        "{\"weights\":{\"match\":1.0}}");
+
+    //json - no feature format check (default to sparse)
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("title:bloomberg");
+    query.add("rows", "10");
+    query.add("fl", "*,score,fv:[fv store=test4 fvwt=json]");
+    query.add("rq", "{!ltr reRankDocs=10 model=sum4}");
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[0]/fv/=={'match':1.0,'c4':1.0}");
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[1]/fv/=={'c4':1.0}");
+
+    //json - sparse feature format check
+    query.remove("fl");
+    query.add("fl", "*,score,fv:[fv store=test4 format=sparse fvwt=json]");
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[0]/fv/=={'match':1.0,'c4':1.0}");
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[1]/fv/=={'c4':1.0}");
+
+    //json - dense feature format check
+    query.remove("fl");
+    query.add("fl", "*,score,fv:[fv store=test4 format=dense fvwt=json]");
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[0]/fv/=={'match':1.0,'c4':1.0}");
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[1]/fv/=={'match':0.0,'c4':1.0}");
+
+    //csv - no feature format check (default to sparse)
+    query.remove("fl");
+    query.add("fl", "*,score,fv:[fv store=test4 fvwt=csv]");
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[0]/fv/=='match:1.0;c4:1.0'");
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[1]/fv/=='c4:1.0'");
+
+    //csv - sparse feature format check
+    query.remove("fl");
+    query.add("fl", "*,score,fv:[fv store=test4 format=sparse fvwt=csv]");
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[0]/fv/=='match:1.0;c4:1.0'");
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[1]/fv/=='c4:1.0'");
+
+    //csv - dense feature format check
+    query.remove("fl");
+    query.add("fl", "*,score,fv:[fv store=test4 format=dense fvwt=csv]");
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[0]/fv/=='match:1.0;c4:1.0'");
+    assertJQ(
+        "/query" + query.toQueryString(),
+        "/response/docs/[1]/fv/=='match:0.0;c4:1.0'");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLtrScoringModel.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLtrScoringModel.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLtrScoringModel.java
new file mode 100644
index 0000000..5fcebad
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLtrScoringModel.java
@@ -0,0 +1,71 @@
+/*
+ * 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.ltr.feature;
+
+import org.apache.solr.ltr.TestRerankBase;
+import org.apache.solr.ltr.store.rest.ManagedFeatureStore;
+import org.apache.solr.ltr.store.rest.TestManagedFeatureStore;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestFeatureLtrScoringModel extends TestRerankBase {
+
+  static ManagedFeatureStore store = null;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    setuptest();
+    store = getManagedFeatureStore();
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    aftertest();
+  }
+
+  @Test
+  public void getInstanceTest() throws FeatureException
+  {
+    store.addFeature(TestManagedFeatureStore.createMap("test",
+        OriginalScoreFeature.class.getCanonicalName(), null),
+        "testFstore");
+    final Feature feature = store.getFeatureStore("testFstore").get("test");
+    assertNotNull(feature);
+    assertEquals("test", feature.getName());
+    assertEquals(OriginalScoreFeature.class.getCanonicalName(), feature
+        .getClass().getCanonicalName());
+  }
+
+  @Test
+  public void getInvalidInstanceTest()
+  {
+    final String nonExistingClassName = "org.apache.solr.ltr.feature.LOLFeature";
+    final ClassNotFoundException expectedException =
+        new ClassNotFoundException(nonExistingClassName);
+    try {
+      store.addFeature(TestManagedFeatureStore.createMap("test",
+          nonExistingClassName, null),
+          "testFstore2");
+      fail("getInvalidInstanceTest failed to throw exception: "+expectedException);
+    } catch (Exception actualException) {
+      Throwable rootError = getRootCause(actualException);
+      assertEquals(expectedException.toString(), rootError.toString());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureStore.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureStore.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureStore.java
new file mode 100644
index 0000000..0ed0cda
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureStore.java
@@ -0,0 +1,106 @@
+/*
+ * 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.ltr.feature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.ltr.TestRerankBase;
+import org.apache.solr.ltr.store.FeatureStore;
+import org.apache.solr.ltr.store.rest.ManagedFeatureStore;
+import org.apache.solr.ltr.store.rest.TestManagedFeatureStore;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestFeatureStore extends TestRerankBase {
+
+  static ManagedFeatureStore fstore = null;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    setuptest();
+    fstore = getManagedFeatureStore();
+  }
+
+  @Test
+  public void testDefaultFeatureStoreName()
+  {
+    assertEquals("_DEFAULT_", FeatureStore.DEFAULT_FEATURE_STORE_NAME);
+    final FeatureStore expectedFeatureStore = fstore.getFeatureStore(FeatureStore.DEFAULT_FEATURE_STORE_NAME);
+    final FeatureStore actualFeatureStore = fstore.getFeatureStore(null);
+    assertEquals("getFeatureStore(null) should return the default feature store", expectedFeatureStore, actualFeatureStore);
+  }
+
+  @Test
+  public void testFeatureStoreAdd() throws FeatureException
+  {
+    final FeatureStore fs = fstore.getFeatureStore("fstore-testFeature");
+    for (int i = 0; i < 5; i++) {
+      final String name = "c" + i;
+
+      fstore.addFeature(TestManagedFeatureStore.createMap(name,
+          OriginalScoreFeature.class.getCanonicalName(), null),
+          "fstore-testFeature");
+
+      final Feature f = fs.get(name);
+      assertNotNull(f);
+
+    }
+    assertEquals(5, fs.getFeatures().size());
+
+  }
+
+  @Test
+  public void testFeatureStoreGet() throws FeatureException
+  {
+    final FeatureStore fs = fstore.getFeatureStore("fstore-testFeature2");
+    for (int i = 0; i < 5; i++) {
+      Map<String,Object> params = new HashMap<String,Object>();
+      params.put("value", i);
+      final String name = "c" + i;
+
+      fstore.addFeature(TestManagedFeatureStore.createMap(name,
+          ValueFeature.class.getCanonicalName(), params),
+          "fstore-testFeature2");
+
+    }
+
+    for (int i = 0; i < 5; i++) {
+      final Feature f = fs.get("c" + i);
+      assertEquals("c" + i, f.getName());
+      assertTrue(f instanceof ValueFeature);
+      final ValueFeature vf = (ValueFeature)f;
+      assertEquals(i, vf.getValue());
+    }
+  }
+
+  @Test
+  public void testMissingFeatureReturnsNull() {
+    final FeatureStore fs = fstore.getFeatureStore("fstore-testFeature3");
+    for (int i = 0; i < 5; i++) {
+      Map<String,Object> params = new HashMap<String,Object>();
+      params.put("value", i);
+      final String name = "testc" + (float) i;
+      fstore.addFeature(TestManagedFeatureStore.createMap(name,
+          ValueFeature.class.getCanonicalName(), params),
+          "fstore-testFeature3");
+
+    }
+    assertNull(fs.get("missing_feature_name"));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldLengthFeature.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldLengthFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldLengthFeature.java
new file mode 100644
index 0000000..4a0d449
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldLengthFeature.java
@@ -0,0 +1,156 @@
+/*
+ * 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.ltr.feature;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.TestRerankBase;
+import org.apache.solr.ltr.model.LinearModel;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestFieldLengthFeature extends TestRerankBase {
+
+  @BeforeClass
+  public static void before() throws Exception {
+    setuptest("solrconfig-ltr.xml", "schema.xml");
+
+    assertU(adoc("id", "1", "title", "w1", "description", "w1"));
+    assertU(adoc("id", "2", "title", "w2 2asd asdd didid", "description",
+        "w2 2asd asdd didid"));
+    assertU(adoc("id", "3", "title", "w3", "description", "w3"));
+    assertU(adoc("id", "4", "title", "w4", "description", "w4"));
+    assertU(adoc("id", "5", "title", "w5", "description", "w5"));
+    assertU(adoc("id", "6", "title", "w1 w2", "description", "w1 w2"));
+    assertU(adoc("id", "7", "title", "w1 w2 w3 w4 w5", "description",
+        "w1 w2 w3 w4 w5 w8"));
+    assertU(adoc("id", "8", "title", "w1 w1 w1 w2 w2 w8", "description",
+        "w1 w1 w1 w2 w2"));
+    assertU(commit());
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    aftertest();
+  }
+
+  @Test
+  public void testIfFieldIsMissingInDocumentLengthIsZero() throws Exception {
+    // add a document without the field 'description'
+    assertU(adoc("id", "42", "title", "w10"));
+    assertU(commit());
+
+    loadFeature("description-length2", FieldLengthFeature.class.getCanonicalName(),
+            "{\"field\":\"description\"}");
+
+    loadModel("description-model2", LinearModel.class.getCanonicalName(),
+            new String[] {"description-length2"}, "{\"weights\":{\"description-length2\":1.0}}");
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("title:w10");
+    query.add("fl", "*, score");
+    query.add("rows", "4");
+    query.add("rq", "{!ltr model=description-model2 reRankDocs=8}");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==0.0");
+  }
+
+
+  @Test
+  public void testIfFieldIsEmptyLengthIsZero() throws Exception {
+    // add a document without the field 'description'
+    assertU(adoc("id", "43", "title", "w11", "description", ""));
+    assertU(commit());
+
+    loadFeature("description-length3", FieldLengthFeature.class.getCanonicalName(),
+            "{\"field\":\"description\"}");
+
+    loadModel("description-model3", LinearModel.class.getCanonicalName(),
+            new String[] {"description-length3"}, "{\"weights\":{\"description-length3\":1.0}}");
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("title:w11");
+    query.add("fl", "*, score");
+    query.add("rows", "4");
+    query.add("rq", "{!ltr model=description-model3 reRankDocs=8}");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==0.0");
+  }
+
+
+  @Test
+  public void testRanking() throws Exception {
+    loadFeature("title-length", FieldLengthFeature.class.getCanonicalName(),
+        "{\"field\":\"title\"}");
+
+    loadModel("title-model", LinearModel.class.getCanonicalName(),
+        new String[] {"title-length"}, "{\"weights\":{\"title-length\":1.0}}");
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("title:w1");
+    query.add("fl", "*, score");
+    query.add("rows", "4");
+
+    // Normal term match
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==4");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='8'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='7'");
+    // Normal term match
+
+    query.add("rq", "{!ltr model=title-model reRankDocs=4}");
+
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==4");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='8'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='7'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='1'");
+
+    query.setQuery("*:*");
+    query.remove("rows");
+    query.add("rows", "8");
+    query.remove("rq");
+    query.add("rq", "{!ltr model=title-model reRankDocs=8}");
+
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='8'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='7'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='2'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='6'");
+
+    loadFeature("description-length",
+        FieldLengthFeature.class.getCanonicalName(),
+        "{\"field\":\"description\"}");
+    loadModel("description-model", LinearModel.class.getCanonicalName(),
+        new String[] {"description-length"},
+        "{\"weights\":{\"description-length\":1.0}}");
+    query.setQuery("title:w1");
+    query.remove("rq");
+    query.remove("rows");
+    query.add("rows", "4");
+    query.add("rq", "{!ltr model=description-model reRankDocs=4}");
+
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==4");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='7'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='8'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='1'");
+  }
+
+
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldValueFeature.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldValueFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldValueFeature.java
new file mode 100644
index 0000000..af150c0
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldValueFeature.java
@@ -0,0 +1,173 @@
+/*
+ * 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.ltr.feature;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.TestRerankBase;
+import org.apache.solr.ltr.model.LinearModel;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestFieldValueFeature extends TestRerankBase {
+
+  private static final float FIELD_VALUE_FEATURE_DEFAULT_VAL = 0.0f;
+
+  @BeforeClass
+  public static void before() throws Exception {
+    setuptest("solrconfig-ltr.xml", "schema.xml");
+
+    assertU(adoc("id", "1", "title", "w1", "description", "w1", "popularity",
+        "1"));
+    assertU(adoc("id", "2", "title", "w2 2asd asdd didid", "description",
+        "w2 2asd asdd didid", "popularity", "2"));
+    assertU(adoc("id", "3", "title", "w3", "description", "w3", "popularity",
+        "3"));
+    assertU(adoc("id", "4", "title", "w4", "description", "w4", "popularity",
+        "4"));
+    assertU(adoc("id", "5", "title", "w5", "description", "w5", "popularity",
+        "5"));
+    assertU(adoc("id", "6", "title", "w1 w2", "description", "w1 w2",
+        "popularity", "6"));
+    assertU(adoc("id", "7", "title", "w1 w2 w3 w4 w5", "description",
+        "w1 w2 w3 w4 w5 w8", "popularity", "7"));
+    assertU(adoc("id", "8", "title", "w1 w1 w1 w2 w2 w8", "description",
+        "w1 w1 w1 w2 w2", "popularity", "8"));
+
+    // a document without the popularity field
+    assertU(adoc("id", "42", "title", "NO popularity", "description", "NO popularity"));
+
+    assertU(commit());
+
+    loadFeature("popularity", FieldValueFeature.class.getCanonicalName(),
+            "{\"field\":\"popularity\"}");
+
+    loadModel("popularity-model", LinearModel.class.getCanonicalName(),
+            new String[] {"popularity"}, "{\"weights\":{\"popularity\":1.0}}");
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    aftertest();
+  }
+
+  @Test
+  public void testRanking() throws Exception {
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("title:w1");
+    query.add("fl", "*, score");
+    query.add("rows", "4");
+
+    // Normal term match
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==4");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='8'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='7'");
+
+    query.add("rq", "{!ltr model=popularity-model reRankDocs=4}");
+
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==4");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='8'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='7'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='1'");
+
+    query.setQuery("*:*");
+    query.remove("rows");
+    query.add("rows", "8");
+    query.remove("rq");
+    query.add("rq", "{!ltr model=popularity-model reRankDocs=8}");
+
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='8'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='7'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='5'");
+  }
+
+
+  @Test
+  public void testIfADocumentDoesntHaveAFieldDefaultValueIsReturned() throws Exception {
+    SolrQuery query = new SolrQuery();
+    query.setQuery("id:42");
+    query.add("fl", "*, score");
+    query.add("rows", "4");
+
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==1");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='42'");
+    query = new SolrQuery();
+    query.setQuery("id:42");
+    query.add("rq", "{!ltr model=popularity-model reRankDocs=4}");
+    query.add("fl", "[fv]");
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==1");
+    assertJQ("/query" + query.toQueryString(),
+            "/response/docs/[0]/=={'[fv]':'popularity:"+FIELD_VALUE_FEATURE_DEFAULT_VAL+"'}");
+
+  }
+
+
+  @Test
+  public void testIfADocumentDoesntHaveAFieldASetDefaultValueIsReturned() throws Exception {
+
+    final String fstore = "testIfADocumentDoesntHaveAFieldASetDefaultValueIsReturned";
+
+    loadFeature("popularity42", FieldValueFeature.class.getCanonicalName(), fstore,
+            "{\"field\":\"popularity\",\"defaultValue\":\"42.0\"}");
+
+    SolrQuery query = new SolrQuery();
+    query.setQuery("id:42");
+    query.add("fl", "*, score");
+    query.add("rows", "4");
+
+    loadModel("popularity-model42", LinearModel.class.getCanonicalName(),
+            new String[] {"popularity42"}, fstore, "{\"weights\":{\"popularity42\":1.0}}");
+
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==1");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='42'");
+    query = new SolrQuery();
+    query.setQuery("id:42");
+    query.add("rq", "{!ltr model=popularity-model42 reRankDocs=4}");
+    query.add("fl", "[fv]");
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==1");
+    assertJQ("/query" + query.toQueryString(),
+            "/response/docs/[0]/=={'[fv]':'popularity42:42.0'}");
+
+  }
+
+  @Test
+  public void testThatIfaFieldDoesNotExistDefaultValueIsReturned() throws Exception {
+    // using a different fstore to avoid a clash with the other tests
+    final String fstore = "testThatIfaFieldDoesNotExistDefaultValueIsReturned";
+    loadFeature("not-existing-field", FieldValueFeature.class.getCanonicalName(), fstore,
+            "{\"field\":\"cowabunga\"}");
+
+    loadModel("not-existing-field-model", LinearModel.class.getCanonicalName(),
+            new String[] {"not-existing-field"}, fstore, "{\"weights\":{\"not-existing-field\":1.0}}");
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("id:42");
+    query.add("rq", "{!ltr model=not-existing-field-model reRankDocs=4}");
+    query.add("fl", "[fv]");
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==1");
+    assertJQ("/query" + query.toQueryString(),
+            "/response/docs/[0]/=={'[fv]':'not-existing-field:"+FIELD_VALUE_FEATURE_DEFAULT_VAL+"'}");
+
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFilterSolrFeature.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFilterSolrFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFilterSolrFeature.java
new file mode 100644
index 0000000..14baefa
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFilterSolrFeature.java
@@ -0,0 +1,105 @@
+/*
+ * 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.ltr.feature;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.TestRerankBase;
+import org.apache.solr.ltr.model.LinearModel;
+import org.apache.solr.ltr.store.rest.ManagedFeatureStore;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestFilterSolrFeature extends TestRerankBase {
+  @BeforeClass
+  public static void before() throws Exception {
+    setuptest("solrconfig-ltr.xml", "schema.xml");
+
+    assertU(adoc("id", "1", "title", "w1", "description", "w1", "popularity",
+        "1"));
+    assertU(adoc("id", "2", "title", "w2 2asd asdd didid", "description",
+        "w2 2asd asdd didid", "popularity", "2"));
+    assertU(adoc("id", "3", "title", "w1", "description", "w1", "popularity",
+        "3"));
+    assertU(adoc("id", "4", "title", "w1", "description", "w1", "popularity",
+        "4"));
+    assertU(adoc("id", "5", "title", "w5", "description", "w5", "popularity",
+        "5"));
+    assertU(adoc("id", "6", "title", "w6 w2", "description", "w1 w2",
+        "popularity", "6"));
+    assertU(adoc("id", "7", "title", "w1 w2 w3 w4 w5", "description",
+        "w6 w2 w3 w4 w5 w8", "popularity", "88888"));
+    assertU(adoc("id", "8", "title", "w1 w1 w1 w2 w2 w8", "description",
+        "w1 w1 w1 w2 w2", "popularity", "88888"));
+    assertU(commit());
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    aftertest();
+  }
+
+  @Test
+  public void testUserTermScoreWithFQ() throws Exception {
+    loadFeature("SomeTermFQ", SolrFeature.class.getCanonicalName(),
+        "{\"fq\":[\"{!terms f=popularity}88888\"]}");
+    loadFeature("SomeEfiFQ", SolrFeature.class.getCanonicalName(),
+        "{\"fq\":[\"{!terms f=title}${user_query}\"]}");
+    loadModel("Term-modelFQ", LinearModel.class.getCanonicalName(),
+        new String[] {"SomeTermFQ", "SomeEfiFQ"},
+        "{\"weights\":{\"SomeTermFQ\":1.6, \"SomeEfiFQ\":2.0}}");
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("fl", "*, score");
+    query.add("rows", "3");
+    query.add("fq", "{!terms f=title}w1");
+    query.add("rq",
+        "{!ltr model=Term-modelFQ reRankDocs=5 efi.user_query='w5'}");
+
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==5");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==3.6");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/score==1.6");
+  }
+
+  @Test
+  public void testBadFeature() throws Exception {
+    // Missing q/fq
+    final String feature = getFeatureInJson("badFeature", "test",
+        SolrFeature.class.getCanonicalName(), "{\"df\":\"foo\"]}");
+    assertJPut(ManagedFeatureStore.REST_END_POINT, feature,
+        "/responseHeader/status==500");
+  }
+
+  @Test
+  public void testFeatureNotEqualWhenNormalizerDifferent() throws Exception {
+    loadFeatures("fq_features.json"); // features that use filter query
+    loadModels("fq-model.json"); // model that uses filter query features
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("fl", "*,score");
+    query.add("rows", "4");
+
+    query.add("rq", "{!ltr reRankDocs=4 model=fqmodel efi.user_query=w2}");
+    query.add("fl", "fv:[fv]");
+
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='2'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='1'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='3'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv=='matchedTitle:1.0;popularity:3.0'");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestNoMatchSolrFeature.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestNoMatchSolrFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestNoMatchSolrFeature.java
new file mode 100644
index 0000000..5712687
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestNoMatchSolrFeature.java
@@ -0,0 +1,192 @@
+/*
+ * 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.ltr.feature;
+
+import java.util.ArrayList;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.TestRerankBase;
+import org.apache.solr.ltr.model.LinearModel;
+import org.apache.solr.ltr.model.MultipleAdditiveTreesModel;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.noggit.ObjectBuilder;
+
+public class TestNoMatchSolrFeature extends TestRerankBase {
+
+  @BeforeClass
+  public static void before() throws Exception {
+    setuptest("solrconfig-ltr.xml", "schema.xml");
+
+    assertU(adoc("id", "1", "title", "w1", "description", "w1", "popularity",
+        "1"));
+    assertU(adoc("id", "2", "title", "w2 2asd asdd didid", "description",
+        "w2 2asd asdd didid", "popularity", "2"));
+    assertU(adoc("id", "3", "title", "w3", "description", "w3", "popularity",
+        "3"));
+    assertU(adoc("id", "4", "title", "w4", "description", "w4", "popularity",
+        "4"));
+    assertU(adoc("id", "5", "title", "w5", "description", "w5", "popularity",
+        "5"));
+    assertU(adoc("id", "6", "title", "w1 w2", "description", "w1 w2",
+        "popularity", "6"));
+    assertU(adoc("id", "7", "title", "w1 w2 w3 w4 w5", "description",
+        "w1 w2 w3 w4 w5 w8", "popularity", "7"));
+    assertU(adoc("id", "8", "title", "w1 w1 w1 w2 w2 w8", "description",
+        "w1 w1 w1 w2 w2", "popularity", "8"));
+    assertU(commit());
+
+    loadFeature("nomatchfeature", SolrFeature.class.getCanonicalName(),
+        "{\"q\":\"foobarbat12345\",\"df\":\"title\"}");
+    loadFeature("yesmatchfeature", SolrFeature.class.getCanonicalName(),
+        "{\"q\":\"w1\",\"df\":\"title\"}");
+    loadFeature("nomatchfeature2", SolrFeature.class.getCanonicalName(),
+        "{\"q\":\"foobarbat12345\",\"df\":\"title\"}");
+    loadModel(
+        "nomatchmodel",
+        LinearModel.class.getCanonicalName(),
+        new String[] {"nomatchfeature", "yesmatchfeature", "nomatchfeature2"},
+        "{\"weights\":{\"nomatchfeature\":1.0,\"yesmatchfeature\":1.1,\"nomatchfeature2\":1.1}}");
+
+    loadFeature("nomatchfeature3", SolrFeature.class.getCanonicalName(),
+        "{\"q\":\"foobarbat12345\",\"df\":\"title\"}");
+    loadModel("nomatchmodel2", LinearModel.class.getCanonicalName(),
+        new String[] {"nomatchfeature3"},
+        "{\"weights\":{\"nomatchfeature3\":1.0}}");
+
+    loadFeature("nomatchfeature4", SolrFeature.class.getCanonicalName(),
+        "noMatchFeaturesStore", "{\"q\":\"foobarbat12345\",\"df\":\"title\"}");
+    loadModel("nomatchmodel3", LinearModel.class.getCanonicalName(),
+        new String[] {"nomatchfeature4"}, "noMatchFeaturesStore",
+        "{\"weights\":{\"nomatchfeature4\":1.0}}");
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    aftertest();
+  }
+
+  @Test
+  public void test2NoMatch1YesMatchFeatureReturnsFvWith1FeatureAndDocScoreScaledByModel() throws Exception {
+    // Tests model with all no matching features but 1
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("fl", "*, score,fv:[fv]");
+    query.add("rows", "4");
+    query.add("fv", "true");
+    query.add("rq", "{!ltr model=nomatchmodel reRankDocs=4}");
+
+    final SolrQuery yesMatchFeatureQuery = new SolrQuery();
+    yesMatchFeatureQuery.setQuery("title:w1");
+    yesMatchFeatureQuery.add("fl", "score");
+    yesMatchFeatureQuery.add("rows", "4");
+    String res = restTestHarness.query("/query"
+        + yesMatchFeatureQuery.toQueryString());
+
+    final Map<String,Object> jsonParse = (Map<String,Object>) ObjectBuilder
+        .fromJSON(res);
+    final Double doc0Score = (Double) ((Map<String,Object>) ((ArrayList<Object>) ((Map<String,Object>) jsonParse
+        .get("response")).get("docs")).get(0)).get("score");
+
+
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score=="
+        + (doc0Score * 1.1));
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/fv=='yesmatchfeature:" + doc0Score + "'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='2'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/score==0.0");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/fv==''");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='3'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/score==0.0");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/fv==''");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='4'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/score==0.0");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/fv==''");
+  }
+
+  @Test
+  public void test1NoMatchFeatureReturnsFvWith1MatchingFeatureFromStoreAndDocWith0Score() throws Exception {
+    // Tests model with all no matching features, but 1 feature store feature matching for extraction
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("fl", "*, score,fv:[fv]");
+    query.add("rows", "4");
+    query.add("rq", "{!ltr model=nomatchmodel2 reRankDocs=4}");
+
+    final SolrQuery yesMatchFeatureQuery = new SolrQuery();
+    yesMatchFeatureQuery.setQuery("title:w1");
+    yesMatchFeatureQuery.add("fl", "score");
+    yesMatchFeatureQuery.add("rows", "4");
+    String res = restTestHarness.query("/query"
+        + yesMatchFeatureQuery.toQueryString());
+
+    final Map<String,Object> jsonParse = (Map<String,Object>) ObjectBuilder
+        .fromJSON(res);
+    final Double doc0Score = (Double) ((Map<String,Object>) ((ArrayList<Object>) ((Map<String,Object>) jsonParse
+        .get("response")).get("docs")).get(0)).get("score");
+
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==0.0");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/fv=='yesmatchfeature:" + doc0Score + "'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/score==0.0");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/fv==''");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/score==0.0");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/fv==''");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/score==0.0");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/fv==''");
+  }
+
+  @Test
+  public void tesOnlyNoMatchFeaturesInStoreAndModelReturnsZeroScore() throws Exception {
+    // Tests model with all no matching features
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("fl", "*, score,fv:[fv]");
+    query.add("rows", "4");
+    query.add("fv", "true");
+    query.add("rq", "{!ltr model=nomatchmodel3 reRankDocs=4}");
+
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==0.0");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv==''");
+  }
+
+  @Test
+  public void tesOnlyNoMatchFeaturesInStoreAndModelReturnsNonzeroScore() throws Exception {
+    // Tests model with all no matching features but expects a non 0 score
+    //  MultipleAdditiveTrees will return scores even for docs without any feature matches
+    loadModel(
+        "nomatchmodel4",
+        MultipleAdditiveTreesModel.class.getCanonicalName(),
+        new String[] {"nomatchfeature4"},
+        "noMatchFeaturesStore",
+        "{\"trees\":[{\"weight\":\"1f\", \"root\":{\"feature\": \"matchedTitle\",\"threshold\": \"0.5f\",\"left\":{\"value\" : \"-10\"},\"right\":{\"value\" : \"9\"}}}]}");
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    query.add("fl", "*, score,fv:[fv]");
+    query.add("rows", "4");
+    query.add("rq", "{!ltr model=nomatchmodel4 reRankDocs=4}");
+
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/score==0.0");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv==''");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a511b30a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java
new file mode 100644
index 0000000..e525891
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java
@@ -0,0 +1,148 @@
+/*
+ * 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.ltr.feature;
+
+import java.util.ArrayList;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.TestRerankBase;
+import org.apache.solr.ltr.model.LinearModel;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.noggit.ObjectBuilder;
+
+public class TestOriginalScoreFeature extends TestRerankBase {
+
+  @BeforeClass
+  public static void before() throws Exception {
+    setuptest("solrconfig-ltr.xml", "schema.xml");
+
+    assertU(adoc("id", "1", "title", "w1"));
+    assertU(adoc("id", "2", "title", "w2"));
+    assertU(adoc("id", "3", "title", "w3"));
+    assertU(adoc("id", "4", "title", "w4"));
+    assertU(adoc("id", "5", "title", "w5"));
+    assertU(adoc("id", "6", "title", "w1 w2"));
+    assertU(adoc("id", "7", "title", "w1 w2 w3 w4 w5"));
+    assertU(adoc("id", "8", "title", "w1 w1 w1 w2 w2"));
+    assertU(commit());
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    aftertest();
+  }
+
+  @Test
+  public void testOriginalScore() throws Exception {
+    loadFeature("score", OriginalScoreFeature.class.getCanonicalName(), "{}");
+
+    loadModel("originalScore", LinearModel.class.getCanonicalName(),
+        new String[] {"score"}, "{\"weights\":{\"score\":1.0}}");
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("title:w1");
+    query.add("fl", "*, score");
+    query.add("rows", "4");
+    query.add("wt", "json");
+
+    // Normal term match
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==4");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='8'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='7'");
+
+    final String res = restTestHarness.query("/query" + query.toQueryString());
+    final Map<String,Object> jsonParse = (Map<String,Object>) ObjectBuilder
+        .fromJSON(res);
+    final String doc0Score = ((Double) ((Map<String,Object>) ((ArrayList<Object>) ((Map<String,Object>) jsonParse
+        .get("response")).get("docs")).get(0)).get("score")).toString();
+    final String doc1Score = ((Double) ((Map<String,Object>) ((ArrayList<Object>) ((Map<String,Object>) jsonParse
+        .get("response")).get("docs")).get(1)).get("score")).toString();
+    final String doc2Score = ((Double) ((Map<String,Object>) ((ArrayList<Object>) ((Map<String,Object>) jsonParse
+        .get("response")).get("docs")).get(2)).get("score")).toString();
+    final String doc3Score = ((Double) ((Map<String,Object>) ((ArrayList<Object>) ((Map<String,Object>) jsonParse
+        .get("response")).get("docs")).get(3)).get("score")).toString();
+
+    query.add("fl", "[fv]");
+    query.add("rq", "{!ltr model=originalScore reRankDocs=4}");
+
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==4");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score=="
+        + doc0Score);
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='8'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/score=="
+        + doc1Score);
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/score=="
+        + doc2Score);
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='7'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/score=="
+        + doc3Score);
+  }
+
+  @Test
+  public void testOriginalScoreWithNonScoringFeatures() throws Exception {
+    loadFeature("origScore", OriginalScoreFeature.class.getCanonicalName(),
+        "store2", "{}");
+    loadFeature("c2", ValueFeature.class.getCanonicalName(), "store2",
+        "{\"value\":2.0}");
+
+    loadModel("origScore", LinearModel.class.getCanonicalName(),
+        new String[] {"origScore"}, "store2",
+        "{\"weights\":{\"origScore\":1.0}}");
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("title:w1");
+    query.add("fl", "*, score, fv:[fv]");
+    query.add("rows", "4");
+    query.add("wt", "json");
+    query.add("rq", "{!ltr model=origScore reRankDocs=4}");
+
+    final String res = restTestHarness.query("/query" + query.toQueryString());
+    final Map<String,Object> jsonParse = (Map<String,Object>) ObjectBuilder
+        .fromJSON(res);
+    final String doc0Score = ((Double) ((Map<String,Object>) ((ArrayList<Object>) ((Map<String,Object>) jsonParse
+        .get("response")).get("docs")).get(0)).get("score")).toString();
+    final String doc1Score = ((Double) ((Map<String,Object>) ((ArrayList<Object>) ((Map<String,Object>) jsonParse
+        .get("response")).get("docs")).get(1)).get("score")).toString();
+    final String doc2Score = ((Double) ((Map<String,Object>) ((ArrayList<Object>) ((Map<String,Object>) jsonParse
+        .get("response")).get("docs")).get(2)).get("score")).toString();
+    final String doc3Score = ((Double) ((Map<String,Object>) ((ArrayList<Object>) ((Map<String,Object>) jsonParse
+        .get("response")).get("docs")).get(3)).get("score")).toString();
+
+    assertJQ("/query" + query.toQueryString(), "/response/numFound/==4");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[0]/fv=='origScore:" + doc0Score + ";c2:2.0'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='8'");
+
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[1]/fv=='origScore:" + doc1Score + ";c2:2.0'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[2]/fv=='origScore:" + doc2Score + ";c2:2.0'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='7'");
+    assertJQ("/query" + query.toQueryString(),
+        "/response/docs/[3]/fv=='origScore:" + doc3Score + ";c2:2.0'");
+  }
+
+}