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:09 UTC

[10/13] lucene-solr:branch_6x: SOLR-8542, SOLR-9746: prefix solr/contrib/ltr's search and response.transform packages with ltr

SOLR-8542, SOLR-9746: prefix solr/contrib/ltr's search and response.transform packages with ltr


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

Branch: refs/heads/branch_6x
Commit: 252c6e9385ba516887543eb1968c8654b35b2b81
Parents: f87d672
Author: Christine Poerschke <cp...@apache.org>
Authored: Fri Nov 11 18:38:36 2016 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Thu Dec 8 18:04:07 2016 +0000

----------------------------------------------------------------------
 solr/contrib/ltr/README.md                      |  10 +-
 solr/contrib/ltr/example/solrconfig.xml         |   4 +-
 .../LTRFeatureLoggerTransformerFactory.java     | 256 +++++++++++++++++++
 .../ltr/response/transform/package-info.java    |  23 ++
 .../solr/ltr/search/LTRQParserPlugin.java       | 241 +++++++++++++++++
 .../apache/solr/ltr/search/package-info.java    |  23 ++
 .../LTRFeatureLoggerTransformerFactory.java     | 254 ------------------
 .../solr/response/transform/package-info.java   |  23 --
 .../apache/solr/search/LTRQParserPlugin.java    | 236 -----------------
 .../org/apache/solr/search/package-info.java    |  23 --
 solr/contrib/ltr/src/java/overview.html         |   6 +-
 .../solr/collection1/conf/solrconfig-ltr.xml    |   4 +-
 .../collection1/conf/solrconfig-ltr_Th10_10.xml |   4 +-
 .../collection1/conf/solrconfig-multiseg.xml    |   6 +-
 .../solr/ltr/store/rest/TestModelManager.java   |   2 +-
 15 files changed, 561 insertions(+), 554 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/README.md
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/README.md b/solr/contrib/ltr/README.md
index 5fe0087..88e2f67 100644
--- a/solr/contrib/ltr/README.md
+++ b/solr/contrib/ltr/README.md
@@ -90,7 +90,7 @@ BONUS: Train an actual machine learning model
   ...
 
   <!-- Query parser used to rerank top docs with a provided model -->
-  <queryParser name="ltr" class="org.apache.solr.search.LTRQParserPlugin" />
+  <queryParser name="ltr" class="org.apache.solr.ltr.search.LTRQParserPlugin" />
 
   <!--  Transformer that will encode the document features in the response.
   For each document the transformer will add the features as an extra field
@@ -99,7 +99,7 @@ BONUS: Train an actual machine learning model
   In order to get the feature vector you will have to
   specify that you want the field (e.g., fl="*,[features])  -->
 
-  <transformer name="features" class="org.apache.solr.response.transform.LTRFeatureLoggerTransformerFactory" />
+  <transformer name="features" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory" />
 
   <query>
     ...
@@ -373,7 +373,7 @@ At this point you'll need to collect feature vectors for each query document pai
 from the Extract features section above to do this. An example script has been included in example/train_and_upload_demo_model.py.
 
 # Explanation of the core reranking logic
-An LTR model is plugged into the ranking through the [LTRQParserPlugin](/solr/contrib/ltr/src/java/org/apache/solr/search/LTRQParserPlugin.java). The plugin will
+An LTR model is plugged into the ranking through the [LTRQParserPlugin](/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/LTRQParserPlugin.java). The plugin will
 read from the request the model, an instance of [LTRScoringModel](/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/LTRScoringModel.java),
 plus other parameters. The plugin will generate an LTRQuery, a particular [ReRankQuery](/solr/core/src/java/org/apache/solr/search/AbstractReRankQuery.java).
 It wraps the original solr query for the first pass ranking, and uses the provided model in an
@@ -388,13 +388,13 @@ About half the time for ranking is spent in the creation of weights for each fea
 
 <config>
   <!-- Query parser used to rerank top docs with a provided model -->
-  <queryParser name="ltr" class="org.apache.solr.search.LTRQParserPlugin">
+  <queryParser name="ltr" class="org.apache.solr.ltr.search.LTRQParserPlugin">
      <int name="threadModule.totalPoolThreads">10</int> <!-- Maximum threads to share for all requests -->
      <int name="threadModule.numThreadsPerRequest">5</int> <!-- Maximum threads to use for a single requests-->
   </queryParser>
   
   <!-- Transformer for extracting features -->
-  <transformer name="features" class="org.apache.solr.response.transform.LTRFeatureLoggerTransformerFactory">
+  <transformer name="features" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory">
      <int name="threadModule.totalPoolThreads">10</int> <!-- Maximum threads to share for all requests -->
      <int name="threadModule.numThreadsPerRequest">5</int> <!-- Maximum threads to use for a single requests-->
   </transformer>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/example/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/solrconfig.xml b/solr/contrib/ltr/example/solrconfig.xml
index 18d6cb8..55da6a4 100644
--- a/solr/contrib/ltr/example/solrconfig.xml
+++ b/solr/contrib/ltr/example/solrconfig.xml
@@ -839,7 +839,7 @@
     </requestHandler>
 
  <!-- Query parser used to rerank top docs with a provided model -->
-  <queryParser name="ltr" class="org.apache.solr.search.LTRQParserPlugin" >
+  <queryParser name="ltr" class="org.apache.solr.ltr.search.LTRQParserPlugin" >
     <int name="threadModule.totalPoolThreads">10</int> <!-- Maximum threads to use for all queries -->
     <int name="threadModule.numThreadsPerRequest">10</int> <!-- Maximum threads to use for a single query-->
   </queryParser>
@@ -848,7 +848,7 @@
   will add the features as an extra field in the response. The name of the field we will be the the name of the
   transformer enclosed between brackets (in this case [features]). In order to get the feature vector you will have to
   specify that you want the field (e.g., fl="*,[features])  -->
-  <transformer name="features" class="org.apache.solr.response.transform.LTRFeatureLoggerTransformerFactory" />
+  <transformer name="features" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory" />
 
 
   <!-- A request handler that returns indented JSON by default -->

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java
new file mode 100644
index 0000000..93ebe63
--- /dev/null
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java
@@ -0,0 +1,256 @@
+/*
+ * 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.response.transform;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.Explanation;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.ltr.FeatureLogger;
+import org.apache.solr.ltr.LTRRescorer;
+import org.apache.solr.ltr.LTRScoringQuery;
+import org.apache.solr.ltr.LTRThreadModule;
+import org.apache.solr.ltr.SolrQueryRequestContextUtils;
+import org.apache.solr.ltr.feature.Feature;
+import org.apache.solr.ltr.model.LTRScoringModel;
+import org.apache.solr.ltr.norm.Normalizer;
+import org.apache.solr.ltr.search.LTRQParserPlugin;
+import org.apache.solr.ltr.store.FeatureStore;
+import org.apache.solr.ltr.store.rest.ManagedFeatureStore;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.ResultContext;
+import org.apache.solr.response.transform.DocTransformer;
+import org.apache.solr.response.transform.TransformerFactory;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.util.SolrPluginUtils;
+
+/**
+ * This transformer will take care to generate and append in the response the
+ * features declared in the feature store of the current model. The class is
+ * useful if you are not interested in the reranking (e.g., bootstrapping a
+ * machine learning framework).
+ */
+public class LTRFeatureLoggerTransformerFactory extends TransformerFactory {
+
+  // used inside fl to specify the output format (csv/json) of the extracted features
+  private static final String FV_RESPONSE_WRITER = "fvwt";
+
+  // used inside fl to specify the format (dense|sparse) of the extracted features
+  private static final String FV_FORMAT = "format";
+
+  // used inside fl to specify the feature store to use for the feature extraction
+  private static final String FV_STORE = "store";
+
+  private static String DEFAULT_LOGGING_MODEL_NAME = "logging-model";
+
+  private String loggingModelName = DEFAULT_LOGGING_MODEL_NAME;
+  private String defaultFvStore;
+  private String defaultFvwt;
+  private String defaultFvFormat;
+
+  private LTRThreadModule threadManager = null;
+
+  public void setLoggingModelName(String loggingModelName) {
+    this.loggingModelName = loggingModelName;
+  }
+
+  public void setStore(String defaultFvStore) {
+    this.defaultFvStore = defaultFvStore;
+  }
+
+  public void setFvwt(String defaultFvwt) {
+    this.defaultFvwt = defaultFvwt;
+  }
+
+  public void setFormat(String defaultFvFormat) {
+    this.defaultFvFormat = defaultFvFormat;
+  }
+
+  @Override
+  public void init(@SuppressWarnings("rawtypes") NamedList args) {
+    super.init(args);
+    threadManager = LTRThreadModule.getInstance(args);
+    SolrPluginUtils.invokeSetters(this, args);
+  }
+
+  @Override
+  public DocTransformer create(String name, SolrParams params,
+      SolrQueryRequest req) {
+
+    // Hint to enable feature vector cache since we are requesting features
+    SolrQueryRequestContextUtils.setIsExtractingFeatures(req);
+
+    // Communicate which feature store we are requesting features for
+    SolrQueryRequestContextUtils.setFvStoreName(req, params.get(FV_STORE, defaultFvStore));
+
+    // Create and supply the feature logger to be used
+    SolrQueryRequestContextUtils.setFeatureLogger(req,
+        FeatureLogger.createFeatureLogger(
+            params.get(FV_RESPONSE_WRITER, defaultFvwt),
+            params.get(FV_FORMAT, defaultFvFormat)));
+
+    return new FeatureTransformer(name, params, req);
+  }
+
+  class FeatureTransformer extends DocTransformer {
+
+    final private String name;
+    final private SolrParams params;
+    final private SolrQueryRequest req;
+
+    private List<LeafReaderContext> leafContexts;
+    private SolrIndexSearcher searcher;
+    private LTRScoringQuery scoringQuery;
+    private LTRScoringQuery.ModelWeight modelWeight;
+    private FeatureLogger<?> featureLogger;
+    private boolean docsWereNotReranked;
+
+    /**
+     * @param name
+     *          Name of the field to be added in a document representing the
+     *          feature vectors
+     */
+    public FeatureTransformer(String name, SolrParams params,
+        SolrQueryRequest req) {
+      this.name = name;
+      this.params = params;
+      this.req = req;
+    }
+
+    @Override
+    public String getName() {
+      return name;
+    }
+
+    @Override
+    public void setContext(ResultContext context) {
+      super.setContext(context);
+      if (context == null) {
+        return;
+      }
+      if (context.getRequest() == null) {
+        return;
+      }
+
+      searcher = context.getSearcher();
+      if (searcher == null) {
+        throw new SolrException(
+            SolrException.ErrorCode.BAD_REQUEST,
+            "searcher is null");
+      }
+      leafContexts = searcher.getTopReaderContext().leaves();
+
+      // Setup LTRScoringQuery
+      scoringQuery = SolrQueryRequestContextUtils.getScoringQuery(req);
+      docsWereNotReranked = (scoringQuery == null);
+      String featureStoreName = SolrQueryRequestContextUtils.getFvStoreName(req);
+      if (docsWereNotReranked || (featureStoreName != null && (!featureStoreName.equals(scoringQuery.getScoringModel().getFeatureStoreName())))) {
+        // if store is set in the transformer we should overwrite the logger
+
+        final ManagedFeatureStore fr = ManagedFeatureStore.getManagedFeatureStore(req.getCore());
+
+        final FeatureStore store = fr.getFeatureStore(featureStoreName);
+        featureStoreName = store.getName(); // if featureStoreName was null before this gets actual name
+
+        try {
+          final LoggingModel lm = new LoggingModel(loggingModelName,
+              featureStoreName, store.getFeatures());
+
+          scoringQuery = new LTRScoringQuery(lm,
+              LTRQParserPlugin.extractEFIParams(params),
+              true,
+              threadManager); // request feature weights to be created for all features
+
+          // Local transformer efi if provided
+          scoringQuery.setOriginalQuery(context.getQuery());
+
+        }catch (final Exception e) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              "retrieving the feature store "+featureStoreName, e);
+        }
+      }
+
+      if (scoringQuery.getFeatureLogger() == null){
+        scoringQuery.setFeatureLogger( SolrQueryRequestContextUtils.getFeatureLogger(req) );
+      }
+      scoringQuery.setRequest(req);
+
+      featureLogger = scoringQuery.getFeatureLogger();
+
+      try {
+        modelWeight = scoringQuery.createWeight(searcher, true, 1f);
+      } catch (final IOException e) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e.getMessage(), e);
+      }
+      if (modelWeight == null) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "error logging the features, model weight is null");
+      }
+    }
+
+    @Override
+    public void transform(SolrDocument doc, int docid, float score)
+        throws IOException {
+      Object fv = featureLogger.getFeatureVector(docid, scoringQuery, searcher);
+      if (fv == null) { // FV for this document was not in the cache
+        fv = featureLogger.makeFeatureVector(
+            LTRRescorer.extractFeaturesInfo(
+                modelWeight,
+                docid,
+                (docsWereNotReranked ? new Float(score) : null),
+                leafContexts));
+      }
+
+      doc.addField(name, fv);
+    }
+
+  }
+
+  private static class LoggingModel extends LTRScoringModel {
+
+    public LoggingModel(String name, String featureStoreName, List<Feature> allFeatures){
+      this(name, Collections.emptyList(), Collections.emptyList(),
+          featureStoreName, allFeatures, Collections.emptyMap());
+    }
+
+    protected LoggingModel(String name, List<Feature> features,
+        List<Normalizer> norms, String featureStoreName,
+        List<Feature> allFeatures, Map<String,Object> params) {
+      super(name, features, norms, featureStoreName, allFeatures, params);
+    }
+
+    @Override
+    public float score(float[] modelFeatureValuesNormalized) {
+      return 0;
+    }
+
+    @Override
+    public Explanation explain(LeafReaderContext context, int doc, float finalScore, List<Explanation> featureExplanations) {
+      return Explanation.match(finalScore, toString()
+          + " logging model, used only for logging the features");
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/package-info.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/package-info.java
new file mode 100644
index 0000000..a946135
--- /dev/null
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * APIs and implementations of {@link org.apache.solr.response.transform.DocTransformer} for modifying documents in Solr request responses
+ */
+package org.apache.solr.ltr.response.transform;
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/LTRQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/LTRQParserPlugin.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/LTRQParserPlugin.java
new file mode 100644
index 0000000..2b4d570
--- /dev/null
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/LTRQParserPlugin.java
@@ -0,0 +1,241 @@
+/*
+ * 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.search;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.ltr.LTRRescorer;
+import org.apache.solr.ltr.LTRScoringQuery;
+import org.apache.solr.ltr.LTRThreadModule;
+import org.apache.solr.ltr.SolrQueryRequestContextUtils;
+import org.apache.solr.ltr.model.LTRScoringModel;
+import org.apache.solr.ltr.store.rest.ManagedFeatureStore;
+import org.apache.solr.ltr.store.rest.ManagedModelStore;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.rest.ManagedResource;
+import org.apache.solr.rest.ManagedResourceObserver;
+import org.apache.solr.search.AbstractReRankQuery;
+import org.apache.solr.search.QParser;
+import org.apache.solr.search.QParserPlugin;
+import org.apache.solr.search.RankQuery;
+import org.apache.solr.search.SyntaxError;
+import org.apache.solr.util.SolrPluginUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Plug into solr a rerank model.
+ *
+ * Learning to Rank Query Parser Syntax: rq={!ltr model=6029760550880411648 reRankDocs=300
+ * efi.myCompanyQueryIntent=0.98}
+ *
+ */
+public class LTRQParserPlugin extends QParserPlugin implements ResourceLoaderAware, ManagedResourceObserver {
+  public static final String NAME = "ltr";
+  private static Query defaultQuery = new MatchAllDocsQuery();
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  // params for setting custom external info that features can use, like query
+  // intent
+  static final String EXTERNAL_FEATURE_INFO = "efi.";
+
+  private ManagedFeatureStore fr = null;
+  private ManagedModelStore mr = null;
+
+  private LTRThreadModule threadManager = null;
+
+  /** query parser plugin: the name of the attribute for setting the model **/
+  public static final String MODEL = "model";
+
+  /** query parser plugin: default number of documents to rerank **/
+  public static final int DEFAULT_RERANK_DOCS = 200;
+
+  /**
+   * query parser plugin:the param that will select how the number of document
+   * to rerank
+   **/
+  public static final String RERANK_DOCS = "reRankDocs";
+
+  @Override
+  public void init(@SuppressWarnings("rawtypes") NamedList args) {
+    super.init(args);
+    threadManager = LTRThreadModule.getInstance(args);
+    SolrPluginUtils.invokeSetters(this, args);
+  }
+
+  @Override
+  public QParser createParser(String qstr, SolrParams localParams,
+      SolrParams params, SolrQueryRequest req) {
+    return new LTRQParser(qstr, localParams, params, req);
+  }
+
+  /**
+   * Given a set of local SolrParams, extract all of the efi.key=value params into a map
+   * @param localParams Local request parameters that might conatin efi params
+   * @return Map of efi params, where the key is the name of the efi param, and the
+   *  value is the value of the efi param
+   */
+  public static Map<String,String[]> extractEFIParams(SolrParams localParams) {
+    final Map<String,String[]> externalFeatureInfo = new HashMap<>();
+    for (final Iterator<String> it = localParams.getParameterNamesIterator(); it
+        .hasNext();) {
+      final String name = it.next();
+      if (name.startsWith(EXTERNAL_FEATURE_INFO)) {
+        externalFeatureInfo.put(
+            name.substring(EXTERNAL_FEATURE_INFO.length()),
+            new String[] {localParams.get(name)});
+      }
+    }
+    return externalFeatureInfo;
+  }
+
+
+  @Override
+  public void inform(ResourceLoader loader) throws IOException {
+    final SolrResourceLoader solrResourceLoader = (SolrResourceLoader) loader;
+    ManagedFeatureStore.registerManagedFeatureStore(solrResourceLoader, this);
+    ManagedModelStore.registerManagedModelStore(solrResourceLoader, this);
+  }
+
+  @Override
+  public void onManagedResourceInitialized(NamedList<?> args, ManagedResource res) throws SolrException {
+    if (res instanceof ManagedFeatureStore) {
+      fr = (ManagedFeatureStore)res;
+    }
+    if (res instanceof ManagedModelStore){
+      mr = (ManagedModelStore)res;
+    }
+    if (mr != null && fr != null){
+      mr.setManagedFeatureStore(fr);
+      // now we can safely load the models
+      mr.loadStoredModels();
+
+    }
+  }
+
+  public class LTRQParser extends QParser {
+
+    public LTRQParser(String qstr, SolrParams localParams, SolrParams params,
+        SolrQueryRequest req) {
+      super(qstr, localParams, params, req);
+    }
+
+    @Override
+    public Query parse() throws SyntaxError {
+      // ReRanking Model
+      final String modelName = localParams.get(LTRQParserPlugin.MODEL);
+      if ((modelName == null) || modelName.isEmpty()) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "Must provide model in the request");
+      }
+
+      final LTRScoringModel ltrScoringModel = mr.getModel(modelName);
+      if (ltrScoringModel == null) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "cannot find " + LTRQParserPlugin.MODEL + " " + modelName);
+      }
+
+      final String modelFeatureStoreName = ltrScoringModel.getFeatureStoreName();
+      final boolean extractFeatures = SolrQueryRequestContextUtils.isExtractingFeatures(req);
+      final String fvStoreName = SolrQueryRequestContextUtils.getFvStoreName(req);
+      // Check if features are requested and if the model feature store and feature-transform feature store are the same
+      final boolean featuresRequestedFromSameStore = (modelFeatureStoreName.equals(fvStoreName) || fvStoreName == null) ? extractFeatures:false;
+
+      final LTRScoringQuery scoringQuery = new LTRScoringQuery(ltrScoringModel,
+          extractEFIParams(localParams),
+          featuresRequestedFromSameStore, threadManager);
+
+      // Enable the feature vector caching if we are extracting features, and the features
+      // we requested are the same ones we are reranking with
+      if (featuresRequestedFromSameStore) {
+        scoringQuery.setFeatureLogger( SolrQueryRequestContextUtils.getFeatureLogger(req) );
+      }
+      SolrQueryRequestContextUtils.setScoringQuery(req, scoringQuery);
+
+      int reRankDocs = localParams.getInt(RERANK_DOCS, DEFAULT_RERANK_DOCS);
+      if (reRankDocs <= 0) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Must rerank at least 1 document");
+      }
+
+      // External features
+      scoringQuery.setRequest(req);
+
+      return new LTRQuery(scoringQuery, reRankDocs);
+    }
+  }
+
+  /**
+   * A learning to rank Query, will incapsulate a learning to rank model, and delegate to it the rescoring
+   * of the documents.
+   **/
+  public class LTRQuery extends AbstractReRankQuery {
+    private final LTRScoringQuery scoringQuery;
+
+    public LTRQuery(LTRScoringQuery scoringQuery, int reRankDocs) {
+      super(defaultQuery, reRankDocs, new LTRRescorer(scoringQuery));
+      this.scoringQuery = scoringQuery;
+    }
+
+    @Override
+    public int hashCode() {
+      return 31 * classHash() + (mainQuery.hashCode() + scoringQuery.hashCode() + reRankDocs);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      return sameClassAs(o) &&  equalsTo(getClass().cast(o));
+    }
+
+    private boolean equalsTo(LTRQuery other) {
+      return (mainQuery.equals(other.mainQuery)
+          && scoringQuery.equals(other.scoringQuery) && (reRankDocs == other.reRankDocs));
+    }
+
+    @Override
+    public RankQuery wrap(Query _mainQuery) {
+      super.wrap(_mainQuery);
+      scoringQuery.setOriginalQuery(_mainQuery);
+      return this;
+    }
+
+    @Override
+    public String toString(String field) {
+      return "{!ltr mainQuery='" + mainQuery.toString() + "' scoringQuery='"
+          + scoringQuery.toString() + "' reRankDocs=" + reRankDocs + "}";
+    }
+
+    @Override
+    protected Query rewrite(Query rewrittenMainQuery) throws IOException {
+      return new LTRQuery(scoringQuery, reRankDocs).wrap(rewrittenMainQuery);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/package-info.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/package-info.java
new file mode 100644
index 0000000..95b645b
--- /dev/null
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * APIs and classes for {@linkplain org.apache.solr.search.QParserPlugin parsing} and {@linkplain org.apache.solr.search.SolrIndexSearcher processing} search requests
+ */
+package org.apache.solr.ltr.search;
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/java/org/apache/solr/response/transform/LTRFeatureLoggerTransformerFactory.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/response/transform/LTRFeatureLoggerTransformerFactory.java b/solr/contrib/ltr/src/java/org/apache/solr/response/transform/LTRFeatureLoggerTransformerFactory.java
deleted file mode 100644
index d144292..0000000
--- a/solr/contrib/ltr/src/java/org/apache/solr/response/transform/LTRFeatureLoggerTransformerFactory.java
+++ /dev/null
@@ -1,254 +0,0 @@
-/*
- * 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.response.transform;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.Explanation;
-import org.apache.solr.common.SolrDocument;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.ltr.FeatureLogger;
-import org.apache.solr.ltr.LTRRescorer;
-import org.apache.solr.ltr.LTRScoringQuery;
-import org.apache.solr.ltr.LTRThreadModule;
-import org.apache.solr.ltr.SolrQueryRequestContextUtils;
-import org.apache.solr.ltr.feature.Feature;
-import org.apache.solr.ltr.model.LTRScoringModel;
-import org.apache.solr.ltr.norm.Normalizer;
-import org.apache.solr.ltr.store.FeatureStore;
-import org.apache.solr.ltr.store.rest.ManagedFeatureStore;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.ResultContext;
-import org.apache.solr.search.LTRQParserPlugin;
-import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.util.SolrPluginUtils;
-
-/**
- * This transformer will take care to generate and append in the response the
- * features declared in the feature store of the current model. The class is
- * useful if you are not interested in the reranking (e.g., bootstrapping a
- * machine learning framework).
- */
-public class LTRFeatureLoggerTransformerFactory extends TransformerFactory {
-
-  // used inside fl to specify the output format (csv/json) of the extracted features
-  private static final String FV_RESPONSE_WRITER = "fvwt";
-
-  // used inside fl to specify the format (dense|sparse) of the extracted features
-  private static final String FV_FORMAT = "format";
-
-  // used inside fl to specify the feature store to use for the feature extraction
-  private static final String FV_STORE = "store";
-
-  private static String DEFAULT_LOGGING_MODEL_NAME = "logging-model";
-
-  private String loggingModelName = DEFAULT_LOGGING_MODEL_NAME;
-  private String defaultFvStore;
-  private String defaultFvwt;
-  private String defaultFvFormat;
-
-  private LTRThreadModule threadManager = null;
-
-  public void setLoggingModelName(String loggingModelName) {
-    this.loggingModelName = loggingModelName;
-  }
-
-  public void setStore(String defaultFvStore) {
-    this.defaultFvStore = defaultFvStore;
-  }
-
-  public void setFvwt(String defaultFvwt) {
-    this.defaultFvwt = defaultFvwt;
-  }
-
-  public void setFormat(String defaultFvFormat) {
-    this.defaultFvFormat = defaultFvFormat;
-  }
-
-  @Override
-  public void init(@SuppressWarnings("rawtypes") NamedList args) {
-    super.init(args);
-    threadManager = LTRThreadModule.getInstance(args);
-    SolrPluginUtils.invokeSetters(this, args);
-  }
-
-  @Override
-  public DocTransformer create(String name, SolrParams params,
-      SolrQueryRequest req) {
-
-    // Hint to enable feature vector cache since we are requesting features
-    SolrQueryRequestContextUtils.setIsExtractingFeatures(req);
-
-    // Communicate which feature store we are requesting features for
-    SolrQueryRequestContextUtils.setFvStoreName(req, params.get(FV_STORE, defaultFvStore));
-
-    // Create and supply the feature logger to be used
-    SolrQueryRequestContextUtils.setFeatureLogger(req,
-        FeatureLogger.createFeatureLogger(
-            params.get(FV_RESPONSE_WRITER, defaultFvwt),
-            params.get(FV_FORMAT, defaultFvFormat)));
-
-    return new FeatureTransformer(name, params, req);
-  }
-
-  class FeatureTransformer extends DocTransformer {
-
-    final private String name;
-    final private SolrParams params;
-    final private SolrQueryRequest req;
-
-    private List<LeafReaderContext> leafContexts;
-    private SolrIndexSearcher searcher;
-    private LTRScoringQuery scoringQuery;
-    private LTRScoringQuery.ModelWeight modelWeight;
-    private FeatureLogger<?> featureLogger;
-    private boolean docsWereNotReranked;
-
-    /**
-     * @param name
-     *          Name of the field to be added in a document representing the
-     *          feature vectors
-     */
-    public FeatureTransformer(String name, SolrParams params,
-        SolrQueryRequest req) {
-      this.name = name;
-      this.params = params;
-      this.req = req;
-    }
-
-    @Override
-    public String getName() {
-      return name;
-    }
-
-    @Override
-    public void setContext(ResultContext context) {
-      super.setContext(context);
-      if (context == null) {
-        return;
-      }
-      if (context.getRequest() == null) {
-        return;
-      }
-
-      searcher = context.getSearcher();
-      if (searcher == null) {
-        throw new SolrException(
-            SolrException.ErrorCode.BAD_REQUEST,
-            "searcher is null");
-      }
-      leafContexts = searcher.getTopReaderContext().leaves();
-
-      // Setup LTRScoringQuery
-      scoringQuery = SolrQueryRequestContextUtils.getScoringQuery(req);
-      docsWereNotReranked = (scoringQuery == null);
-      String featureStoreName = SolrQueryRequestContextUtils.getFvStoreName(req);
-      if (docsWereNotReranked || (featureStoreName != null && (!featureStoreName.equals(scoringQuery.getScoringModel().getFeatureStoreName())))) {
-        // if store is set in the transformer we should overwrite the logger
-
-        final ManagedFeatureStore fr = ManagedFeatureStore.getManagedFeatureStore(req.getCore());
-
-        final FeatureStore store = fr.getFeatureStore(featureStoreName);
-        featureStoreName = store.getName(); // if featureStoreName was null before this gets actual name
-
-        try {
-          final LoggingModel lm = new LoggingModel(loggingModelName,
-              featureStoreName, store.getFeatures());
-
-          scoringQuery = new LTRScoringQuery(lm,
-              LTRQParserPlugin.extractEFIParams(params),
-              true,
-              threadManager); // request feature weights to be created for all features
-
-          // Local transformer efi if provided
-          scoringQuery.setOriginalQuery(context.getQuery());
-
-        }catch (final Exception e) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "retrieving the feature store "+featureStoreName, e);
-        }
-      }
-
-      if (scoringQuery.getFeatureLogger() == null){
-        scoringQuery.setFeatureLogger( SolrQueryRequestContextUtils.getFeatureLogger(req) );
-      }
-      scoringQuery.setRequest(req);
-
-      featureLogger = scoringQuery.getFeatureLogger();
-
-      try {
-        modelWeight = scoringQuery.createWeight(searcher, true, 1f);
-      } catch (final IOException e) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e.getMessage(), e);
-      }
-      if (modelWeight == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "error logging the features, model weight is null");
-      }
-    }
-
-    @Override
-    public void transform(SolrDocument doc, int docid, float score)
-        throws IOException {
-      Object fv = featureLogger.getFeatureVector(docid, scoringQuery, searcher);
-      if (fv == null) { // FV for this document was not in the cache
-        fv = featureLogger.makeFeatureVector(
-            LTRRescorer.extractFeaturesInfo(
-                modelWeight,
-                docid,
-                (docsWereNotReranked ? new Float(score) : null),
-                leafContexts));
-      }
-
-      doc.addField(name, fv);
-    }
-
-  }
-
-  private static class LoggingModel extends LTRScoringModel {
-
-    public LoggingModel(String name, String featureStoreName, List<Feature> allFeatures){
-      this(name, Collections.emptyList(), Collections.emptyList(),
-          featureStoreName, allFeatures, Collections.emptyMap());
-    }
-
-    protected LoggingModel(String name, List<Feature> features,
-        List<Normalizer> norms, String featureStoreName,
-        List<Feature> allFeatures, Map<String,Object> params) {
-      super(name, features, norms, featureStoreName, allFeatures, params);
-    }
-
-    @Override
-    public float score(float[] modelFeatureValuesNormalized) {
-      return 0;
-    }
-
-    @Override
-    public Explanation explain(LeafReaderContext context, int doc, float finalScore, List<Explanation> featureExplanations) {
-      return Explanation.match(finalScore, toString()
-          + " logging model, used only for logging the features");
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/java/org/apache/solr/response/transform/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/response/transform/package-info.java b/solr/contrib/ltr/src/java/org/apache/solr/response/transform/package-info.java
deleted file mode 100644
index bab3ebf..0000000
--- a/solr/contrib/ltr/src/java/org/apache/solr/response/transform/package-info.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * APIs and implementations of {@link org.apache.solr.response.transform.DocTransformer} for modifying documents in Solr request responses
- */
-package org.apache.solr.response.transform;
-
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/java/org/apache/solr/search/LTRQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/search/LTRQParserPlugin.java b/solr/contrib/ltr/src/java/org/apache/solr/search/LTRQParserPlugin.java
deleted file mode 100644
index 8e92638..0000000
--- a/solr/contrib/ltr/src/java/org/apache/solr/search/LTRQParserPlugin.java
+++ /dev/null
@@ -1,236 +0,0 @@
-/*
- * 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.lang.invoke.MethodHandles;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.lucene.analysis.util.ResourceLoader;
-import org.apache.lucene.analysis.util.ResourceLoaderAware;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.ltr.LTRRescorer;
-import org.apache.solr.ltr.LTRScoringQuery;
-import org.apache.solr.ltr.LTRThreadModule;
-import org.apache.solr.ltr.SolrQueryRequestContextUtils;
-import org.apache.solr.ltr.model.LTRScoringModel;
-import org.apache.solr.ltr.store.rest.ManagedFeatureStore;
-import org.apache.solr.ltr.store.rest.ManagedModelStore;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.rest.ManagedResource;
-import org.apache.solr.rest.ManagedResourceObserver;
-import org.apache.solr.util.SolrPluginUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Plug into solr a rerank model.
- *
- * Learning to Rank Query Parser Syntax: rq={!ltr model=6029760550880411648 reRankDocs=300
- * efi.myCompanyQueryIntent=0.98}
- *
- */
-public class LTRQParserPlugin extends QParserPlugin implements ResourceLoaderAware, ManagedResourceObserver {
-  public static final String NAME = "ltr";
-  private static Query defaultQuery = new MatchAllDocsQuery();
-
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  // params for setting custom external info that features can use, like query
-  // intent
-  static final String EXTERNAL_FEATURE_INFO = "efi.";
-
-  private ManagedFeatureStore fr = null;
-  private ManagedModelStore mr = null;
-
-  private LTRThreadModule threadManager = null;
-
-  /** query parser plugin: the name of the attribute for setting the model **/
-  public static final String MODEL = "model";
-
-  /** query parser plugin: default number of documents to rerank **/
-  public static final int DEFAULT_RERANK_DOCS = 200;
-
-  /**
-   * query parser plugin:the param that will select how the number of document
-   * to rerank
-   **/
-  public static final String RERANK_DOCS = "reRankDocs";
-
-  @Override
-  public void init(@SuppressWarnings("rawtypes") NamedList args) {
-    super.init(args);
-    threadManager = LTRThreadModule.getInstance(args);
-    SolrPluginUtils.invokeSetters(this, args);
-  }
-
-  @Override
-  public QParser createParser(String qstr, SolrParams localParams,
-      SolrParams params, SolrQueryRequest req) {
-    return new LTRQParser(qstr, localParams, params, req);
-  }
-
-  /**
-   * Given a set of local SolrParams, extract all of the efi.key=value params into a map
-   * @param localParams Local request parameters that might conatin efi params
-   * @return Map of efi params, where the key is the name of the efi param, and the
-   *  value is the value of the efi param
-   */
-  public static Map<String,String[]> extractEFIParams(SolrParams localParams) {
-    final Map<String,String[]> externalFeatureInfo = new HashMap<>();
-    for (final Iterator<String> it = localParams.getParameterNamesIterator(); it
-        .hasNext();) {
-      final String name = it.next();
-      if (name.startsWith(EXTERNAL_FEATURE_INFO)) {
-        externalFeatureInfo.put(
-            name.substring(EXTERNAL_FEATURE_INFO.length()),
-            new String[] {localParams.get(name)});
-      }
-    }
-    return externalFeatureInfo;
-  }
-
-
-  @Override
-  public void inform(ResourceLoader loader) throws IOException {
-    final SolrResourceLoader solrResourceLoader = (SolrResourceLoader) loader;
-    ManagedFeatureStore.registerManagedFeatureStore(solrResourceLoader, this);
-    ManagedModelStore.registerManagedModelStore(solrResourceLoader, this);
-  }
-
-  @Override
-  public void onManagedResourceInitialized(NamedList<?> args, ManagedResource res) throws SolrException {
-    if (res instanceof ManagedFeatureStore) {
-      fr = (ManagedFeatureStore)res;
-    }
-    if (res instanceof ManagedModelStore){
-      mr = (ManagedModelStore)res;
-    }
-    if (mr != null && fr != null){
-      mr.setManagedFeatureStore(fr);
-      // now we can safely load the models
-      mr.loadStoredModels();
-
-    }
-  }
-
-  public class LTRQParser extends QParser {
-
-    public LTRQParser(String qstr, SolrParams localParams, SolrParams params,
-        SolrQueryRequest req) {
-      super(qstr, localParams, params, req);
-    }
-
-    @Override
-    public Query parse() throws SyntaxError {
-      // ReRanking Model
-      final String modelName = localParams.get(LTRQParserPlugin.MODEL);
-      if ((modelName == null) || modelName.isEmpty()) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "Must provide model in the request");
-      }
-
-      final LTRScoringModel ltrScoringModel = mr.getModel(modelName);
-      if (ltrScoringModel == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "cannot find " + LTRQParserPlugin.MODEL + " " + modelName);
-      }
-
-      final String modelFeatureStoreName = ltrScoringModel.getFeatureStoreName();
-      final boolean extractFeatures = SolrQueryRequestContextUtils.isExtractingFeatures(req);
-      final String fvStoreName = SolrQueryRequestContextUtils.getFvStoreName(req);
-      // Check if features are requested and if the model feature store and feature-transform feature store are the same
-      final boolean featuresRequestedFromSameStore = (modelFeatureStoreName.equals(fvStoreName) || fvStoreName == null) ? extractFeatures:false;
-
-      final LTRScoringQuery scoringQuery = new LTRScoringQuery(ltrScoringModel,
-          extractEFIParams(localParams),
-          featuresRequestedFromSameStore, threadManager);
-
-      // Enable the feature vector caching if we are extracting features, and the features
-      // we requested are the same ones we are reranking with
-      if (featuresRequestedFromSameStore) {
-        scoringQuery.setFeatureLogger( SolrQueryRequestContextUtils.getFeatureLogger(req) );
-      }
-      SolrQueryRequestContextUtils.setScoringQuery(req, scoringQuery);
-
-      int reRankDocs = localParams.getInt(RERANK_DOCS, DEFAULT_RERANK_DOCS);
-      if (reRankDocs <= 0) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Must rerank at least 1 document");
-      }
-
-      // External features
-      scoringQuery.setRequest(req);
-
-      return new LTRQuery(scoringQuery, reRankDocs);
-    }
-  }
-
-  /**
-   * A learning to rank Query, will incapsulate a learning to rank model, and delegate to it the rescoring
-   * of the documents.
-   **/
-  public class LTRQuery extends AbstractReRankQuery {
-    private final LTRScoringQuery scoringQuery;
-
-    public LTRQuery(LTRScoringQuery scoringQuery, int reRankDocs) {
-      super(defaultQuery, reRankDocs, new LTRRescorer(scoringQuery));
-      this.scoringQuery = scoringQuery;
-    }
-
-    @Override
-    public int hashCode() {
-      return 31 * classHash() + (mainQuery.hashCode() + scoringQuery.hashCode() + reRankDocs);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      return sameClassAs(o) &&  equalsTo(getClass().cast(o));
-    }
-
-    private boolean equalsTo(LTRQuery other) {
-      return (mainQuery.equals(other.mainQuery)
-          && scoringQuery.equals(other.scoringQuery) && (reRankDocs == other.reRankDocs));
-    }
-
-    @Override
-    public RankQuery wrap(Query _mainQuery) {
-      super.wrap(_mainQuery);
-      scoringQuery.setOriginalQuery(_mainQuery);
-      return this;
-    }
-
-    @Override
-    public String toString(String field) {
-      return "{!ltr mainQuery='" + mainQuery.toString() + "' scoringQuery='"
-          + scoringQuery.toString() + "' reRankDocs=" + reRankDocs + "}";
-    }
-
-    @Override
-    protected Query rewrite(Query rewrittenMainQuery) throws IOException {
-      return new LTRQuery(scoringQuery, reRankDocs).wrap(rewrittenMainQuery);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/java/org/apache/solr/search/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/search/package-info.java b/solr/contrib/ltr/src/java/org/apache/solr/search/package-info.java
deleted file mode 100644
index 2286a93..0000000
--- a/solr/contrib/ltr/src/java/org/apache/solr/search/package-info.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * APIs and classes for {@linkplain org.apache.solr.search.QParserPlugin parsing} and {@linkplain org.apache.solr.search.SolrIndexSearcher processing} search requests
- */
-package org.apache.solr.search;
-
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/java/overview.html
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/overview.html b/solr/contrib/ltr/src/java/overview.html
index ccae361..d1a22f0 100644
--- a/solr/contrib/ltr/src/java/overview.html
+++ b/solr/contrib/ltr/src/java/overview.html
@@ -35,11 +35,11 @@ feature engineering and feature extraction.
 </p>
 <h2> Code structure </h2>
 <p>
-A Learning to Rank model is plugged into the ranking through the {@link org.apache.solr.search.LTRQParserPlugin},
+A Learning to Rank model is plugged into the ranking through the {@link org.apache.solr.ltr.search.LTRQParserPlugin},
 a {@link org.apache.solr.search.QParserPlugin}. The plugin will
 read from the request the model (instance of {@link org.apache.solr.ltr.model.LTRScoringModel})
 used to perform the request plus other
-parameters. The plugin will generate a {@link org.apache.solr.search.LTRQParserPlugin.LTRQuery LTRQuery}:
+parameters. The plugin will generate a {@link org.apache.solr.ltr.search.LTRQParserPlugin.LTRQuery LTRQuery}:
 a particular {@link org.apache.solr.search.RankQuery}
 that will encapsulate the given model and use it to
 rescore and rerank the document (by using an {@link org.apache.solr.ltr.LTRRescorer}).
@@ -73,7 +73,7 @@ of features. One benefit of extending the Query object is that we can reuse
 Query as a feature, see for example {@link org.apache.solr.ltr.feature.SolrFeature}.
 Features for a document can also be returned in the response by
 using the FeatureTransformer (a {@link org.apache.solr.response.transform.DocTransformer DocTransformer})
-provided by {@link org.apache.solr.response.transform.LTRFeatureLoggerTransformerFactory}.
+provided by {@link org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory}.
 </p>
 <p>
 {@link org.apache.solr.ltr.store} contains all the logic to store all the features and the models.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml
index 1a18471..d6f8d8e 100644
--- a/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml
+++ b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml
@@ -21,7 +21,7 @@
 
  <!-- Query parser used to rerank top docs with a provided model -->
  <queryParser name="ltr"
-  class="org.apache.solr.search.LTRQParserPlugin" />
+  class="org.apache.solr.ltr.search.LTRQParserPlugin" />
 
  <query>
   <filterCache class="solr.FastLRUCache" size="4096"
@@ -36,7 +36,7 @@
   enclosed between brackets (in this case [fv]). In order to get the feature
   vector you will have to specify that you want the field (e.g., fl="*,[fv]) -->
  <transformer name="fv"
-  class="org.apache.solr.response.transform.LTRFeatureLoggerTransformerFactory" />
+  class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory" />
 
  <updateHandler class="solr.DirectUpdateHandler2">
   <autoCommit>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml
index fd0940a..51971f7 100644
--- a/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml
+++ b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml
@@ -20,7 +20,7 @@
 
 
  <!-- Query parser used to rerank top docs with a provided model -->
- <queryParser name="ltr" class="org.apache.solr.search.LTRQParserPlugin" >
+ <queryParser name="ltr" class="org.apache.solr.ltr.search.LTRQParserPlugin" >
   <int name="threadModule.totalPoolThreads">10</int> <!-- Maximum threads to use for all queries -->
   <int name="threadModule.numThreadsPerRequest">10</int> <!-- Maximum threads to use for a single query-->
  </queryParser>
@@ -40,7 +40,7 @@
   enclosed between brackets (in this case [fv]). In order to get the feature
   vector you will have to specify that you want the field (e.g., fl="*,[fv]) -->
  <transformer name="fv"
-  class="org.apache.solr.response.transform.LTRFeatureLoggerTransformerFactory" />
+  class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory" />
 
  <updateHandler class="solr.DirectUpdateHandler2">
   <autoCommit>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml
index a36c1df..5abff6f 100644
--- a/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml
+++ b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml
@@ -20,7 +20,7 @@
 
 
  <!-- Query parser used to rerank top docs with a provided model -->
- <queryParser name="ltr" class="org.apache.solr.search.LTRQParserPlugin" />
+ <queryParser name="ltr" class="org.apache.solr.ltr.search.LTRQParserPlugin" />
 
  <maxBufferedDocs>1</maxBufferedDocs>
  <mergePolicyFactory class="org.apache.solr.index.TieredMergePolicyFactory">
@@ -33,7 +33,7 @@
   enclosed between brackets (in this case [fv]). In order to get the feature 
   vector you will have to specify that you want the field (e.g., fl="*,[fv]) -->
  <transformer name="features"
-  class="org.apache.solr.response.transform.LTRFeatureLoggerTransformerFactory" />
+  class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory" />
 
  <updateHandler class="solr.DirectUpdateHandler2">
   <autoCommit>
@@ -59,4 +59,4 @@
   </lst>
  </requestHandler>
 
-</config>
\ No newline at end of file
+</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/252c6e93/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManager.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManager.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManager.java
index 855f053..8d11a90 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManager.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManager.java
@@ -22,10 +22,10 @@ import org.apache.solr.ltr.TestRerankBase;
 import org.apache.solr.ltr.feature.FieldValueFeature;
 import org.apache.solr.ltr.feature.ValueFeature;
 import org.apache.solr.ltr.model.LinearModel;
+import org.apache.solr.ltr.search.LTRQParserPlugin;
 import org.apache.solr.rest.ManagedResource;
 import org.apache.solr.rest.ManagedResourceStorage;
 import org.apache.solr.rest.RestManager;
-import org.apache.solr.search.LTRQParserPlugin;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;