You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by GitBox <gi...@apache.org> on 2021/06/05 20:08:25 UTC

[GitHub] [solr] cpoerschke commented on a change in pull request #166: SOLR-15450 add PrefetchingFieldValueFeature

cpoerschke commented on a change in pull request #166:
URL: https://github.com/apache/solr/pull/166#discussion_r646023649



##########
File path: solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/PrefetchingFieldValueFeature.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.SolrDocumentFetcher;
+import org.apache.solr.search.SolrIndexSearcher;
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This feature returns the value of a field in the current document.
+ * The field must have stored="true" or docValues="true" properties.
+ * Example configuration:
+ * <pre>{
+  "name":  "rawHits",
+  "class": "org.apache.solr.ltr.feature.FieldValueFeature",

Review comment:
       ```suggestion
     "class": "org.apache.solr.ltr.feature.PrefetchingFieldValueFeature",
   ```

##########
File path: solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/PrefetchingFieldValueFeature.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.SolrDocumentFetcher;
+import org.apache.solr.search.SolrIndexSearcher;
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This feature returns the value of a field in the current document.
+ * The field must have stored="true" or docValues="true" properties.
+ * Example configuration:
+ * <pre>{
+  "name":  "rawHits",
+  "class": "org.apache.solr.ltr.feature.FieldValueFeature",
+  "params": {
+      "field": "hits"
+  }
+}</pre>
+ */
+public class PrefetchingFieldValueFeature extends FieldValueFeature {
+  // used to store all fields from all PrefetchingFieldValueFeatures
+  private Set<String> prefetchFields;
+
+  public void setField(String field) {
+    this.field = field;
+  }
+
+  public void setPrefetchFields(Set<String> fields) {
+    prefetchFields = fields;
+  }
+
+  @Override
+  public LinkedHashMap<String,Object> paramsToMap() {
+    final LinkedHashMap<String,Object> params = defaultParamsToMap();
+    params.put("field", field);
+    return params;
+  }
+
+  @Override
+  protected void validate() throws FeatureException {
+    if (field == null || field.isEmpty()) {
+      throw new FeatureException(getClass().getSimpleName()+
+          ": field must be provided");
+    }
+  }
+
+  public PrefetchingFieldValueFeature(String name, Map<String,Object> params) {
+    super(name, params);
+  }
+
+  @Override
+  public FeatureWeight createWeight(IndexSearcher searcher, boolean needsScores,
+      SolrQueryRequest request, Query originalQuery, Map<String,String[]> efi)
+          throws IOException {
+    return new PrefetchingFieldValueFeatureWeight(searcher, request, originalQuery, efi);
+  }
+
+  @VisibleForTesting
+  public Set<String> getPrefetchFields(){
+    return prefetchFields;
+  }

Review comment:
       minor/subjective: place getter right after setter in the code, easier to see then that both getter and setter are present.

##########
File path: solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/FieldValueFeature.java
##########
@@ -65,7 +65,7 @@
  */
 public class FieldValueFeature extends Feature {
 
-  private String field;
+  String field;

Review comment:
       ```suggestion
     protected String field;
   ```
   
   or if not needed for access in derived class (please see comment below) then keep as private
   
   ```suggestion
     private String field;
   ```

##########
File path: solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/FieldValueFeature.java
##########
@@ -166,33 +166,36 @@ public float score() throws IOException {
         try {
           final Document document = context.reader().document(itr.docID(),
               fieldAsSet);
-          final IndexableField indexableField = document.getField(field);
-          if (indexableField == null) {
-            return getDefaultValue();
-          }
-          final Number number = indexableField.numericValue();
-          if (number != null) {
-            return number.floatValue();
-          } else {
-            final String string = indexableField.stringValue();
-            if (string.length() == 1) {
-              // boolean values in the index are encoded with the
-              // a single char contained in TRUE_TOKEN or FALSE_TOKEN
-              // (see BoolField)
-              if (string.charAt(0) == BoolField.TRUE_TOKEN[0]) {
-                return 1;
-              }
-              if (string.charAt(0) == BoolField.FALSE_TOKEN[0]) {
-                return 0;
-              }
-            }
-          }
+          return parseStoredFieldValue(document.getField(field));
         } catch (final IOException e) {
           throw new FeatureException(
               e.toString() + ": " +
                   "Unable to extract feature for "
                   + name, e);
         }
+      }
+
+      float parseStoredFieldValue(IndexableField indexableField) throws IOException {

Review comment:
       ```suggestion
         protected float parseStoredFieldValue(IndexableField indexableField) throws IOException {
   ```

##########
File path: solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/PrefetchingFieldValueFeature.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.SolrDocumentFetcher;
+import org.apache.solr.search.SolrIndexSearcher;
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This feature returns the value of a field in the current document.
+ * The field must have stored="true" or docValues="true" properties.
+ * Example configuration:
+ * <pre>{
+  "name":  "rawHits",
+  "class": "org.apache.solr.ltr.feature.FieldValueFeature",
+  "params": {
+      "field": "hits"
+  }
+}</pre>
+ */
+public class PrefetchingFieldValueFeature extends FieldValueFeature {
+  // used to store all fields from all PrefetchingFieldValueFeatures
+  private Set<String> prefetchFields;
+
+  public void setField(String field) {
+    this.field = field;
+  }
+
+  public void setPrefetchFields(Set<String> fields) {
+    prefetchFields = fields;
+  }
+
+  @Override
+  public LinkedHashMap<String,Object> paramsToMap() {
+    final LinkedHashMap<String,Object> params = defaultParamsToMap();
+    params.put("field", field);
+    return params;
+  }
+
+  @Override
+  protected void validate() throws FeatureException {
+    if (field == null || field.isEmpty()) {
+      throw new FeatureException(getClass().getSimpleName()+
+          ": field must be provided");
+    }
+  }

Review comment:
       ```suggestion
   ```
   
   Use of super class' methods should work or if not let's add comment w.r.t. why we are overriding.

##########
File path: solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml
##########
@@ -34,6 +34,8 @@
    initialSize="2048" autowarmCount="0" />
   <cache name="QUERY_DOC_FV" class="solr.search.CaffeineCache" size="4096"
    initialSize="2048" autowarmCount="4096" regenerator="solr.search.NoOpRegenerator" />
+  <!-- false is the default, just here to make it explizit in contrast to solrconfig-ltr-lazy-->
+  <enableLazyFieldLoading>false</enableLazyFieldLoading>

Review comment:
       Thanks for adding the comment here!
   
   Question: is the `enableLazyFieldLoading` value the only difference between the `solrconfig-ltr.xml` and `solrconfig-ltr-lazy.xml` files?
   
   Context for the question: if it is the only difference (and no other future differences are anticipated or intended) then we could consider use of something like `${enableLazyFieldLoading:false}` to achieve differentiated test behaviour from a single file. https://github.com/apache/lucene-solr/blob/releases/lucene-solr/8.8.2/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml#L56 does something like that elsewhere but for this boolean or https://github.com/apache/lucene-solr/blob/releases/lucene-solr/8.8.2/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml#L45 does something similar within the LTR tests.

##########
File path: solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/PrefetchingFieldValueFeature.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.SolrDocumentFetcher;
+import org.apache.solr.search.SolrIndexSearcher;
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This feature returns the value of a field in the current document.
+ * The field must have stored="true" or docValues="true" properties.
+ * Example configuration:
+ * <pre>{
+  "name":  "rawHits",
+  "class": "org.apache.solr.ltr.feature.FieldValueFeature",
+  "params": {
+      "field": "hits"
+  }
+}</pre>
+ */
+public class PrefetchingFieldValueFeature extends FieldValueFeature {
+  // used to store all fields from all PrefetchingFieldValueFeatures
+  private Set<String> prefetchFields;
+
+  public void setField(String field) {
+    this.field = field;
+  }
+
+  public void setPrefetchFields(Set<String> fields) {
+    prefetchFields = fields;
+  }
+
+  @Override
+  public LinkedHashMap<String,Object> paramsToMap() {
+    final LinkedHashMap<String,Object> params = defaultParamsToMap();
+    params.put("field", field);
+    return params;
+  }
+
+  @Override
+  protected void validate() throws FeatureException {
+    if (field == null || field.isEmpty()) {
+      throw new FeatureException(getClass().getSimpleName()+
+          ": field must be provided");
+    }
+  }
+
+  public PrefetchingFieldValueFeature(String name, Map<String,Object> params) {
+    super(name, params);
+  }
+
+  @Override
+  public FeatureWeight createWeight(IndexSearcher searcher, boolean needsScores,
+      SolrQueryRequest request, Query originalQuery, Map<String,String[]> efi)
+          throws IOException {
+    return new PrefetchingFieldValueFeatureWeight(searcher, request, originalQuery, efi);
+  }
+
+  @VisibleForTesting
+  public Set<String> getPrefetchFields(){
+    return prefetchFields;
+  }
+
+  public class PrefetchingFieldValueFeatureWeight extends FieldValueFeatureWeight {
+    private final SchemaField schemaField;

Review comment:
       ```suggestion
   ```
   
   If we change the super class's `schemaField` visibility from `private` to `protected` then `PrefetchingFieldValueFeatureWeight` would have access for use without needing to initialise it in the `PrefetchingFieldValueFeatureWeight` constructor.

##########
File path: solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/PrefetchingFieldValueFeature.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.SolrDocumentFetcher;
+import org.apache.solr.search.SolrIndexSearcher;
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This feature returns the value of a field in the current document.
+ * The field must have stored="true" or docValues="true" properties.
+ * Example configuration:
+ * <pre>{
+  "name":  "rawHits",
+  "class": "org.apache.solr.ltr.feature.FieldValueFeature",
+  "params": {
+      "field": "hits"
+  }
+}</pre>
+ */
+public class PrefetchingFieldValueFeature extends FieldValueFeature {
+  // used to store all fields from all PrefetchingFieldValueFeatures
+  private Set<String> prefetchFields;
+
+  public void setField(String field) {
+    this.field = field;
+  }
+
+  public void setPrefetchFields(Set<String> fields) {
+    prefetchFields = fields;
+  }
+
+  @Override
+  public LinkedHashMap<String,Object> paramsToMap() {
+    final LinkedHashMap<String,Object> params = defaultParamsToMap();
+    params.put("field", field);
+    return params;
+  }
+
+  @Override
+  protected void validate() throws FeatureException {
+    if (field == null || field.isEmpty()) {
+      throw new FeatureException(getClass().getSimpleName()+
+          ": field must be provided");
+    }
+  }
+
+  public PrefetchingFieldValueFeature(String name, Map<String,Object> params) {
+    super(name, params);
+  }
+
+  @Override
+  public FeatureWeight createWeight(IndexSearcher searcher, boolean needsScores,
+      SolrQueryRequest request, Query originalQuery, Map<String,String[]> efi)
+          throws IOException {
+    return new PrefetchingFieldValueFeatureWeight(searcher, request, originalQuery, efi);
+  }
+
+  @VisibleForTesting
+  public Set<String> getPrefetchFields(){
+    return prefetchFields;
+  }
+
+  public class PrefetchingFieldValueFeatureWeight extends FieldValueFeatureWeight {
+    private final SchemaField schemaField;
+    private final SolrDocumentFetcher docFetcher;
+
+    public PrefetchingFieldValueFeatureWeight(IndexSearcher searcher,
+        SolrQueryRequest request, Query originalQuery, Map<String,String[]> efi) {
+      super(searcher, request, originalQuery, efi);
+      if (searcher instanceof SolrIndexSearcher) {
+        schemaField = ((SolrIndexSearcher) searcher).getSchema().getFieldOrNull(field);
+      } else {
+        schemaField = null;
+      }
+      this.docFetcher = request.getSearcher().getDocFetcher();

Review comment:
       A comment here w.r.t. why `request.getSearcher()` rather than `(SolrIndexSearcher) searcher` could be useful for future readers of the code.

##########
File path: solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/PrefetchingFieldValueFeature.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.SolrDocumentFetcher;
+import org.apache.solr.search.SolrIndexSearcher;
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This feature returns the value of a field in the current document.
+ * The field must have stored="true" or docValues="true" properties.
+ * Example configuration:
+ * <pre>{
+  "name":  "rawHits",
+  "class": "org.apache.solr.ltr.feature.FieldValueFeature",
+  "params": {
+      "field": "hits"
+  }
+}</pre>
+ */
+public class PrefetchingFieldValueFeature extends FieldValueFeature {
+  // used to store all fields from all PrefetchingFieldValueFeatures
+  private Set<String> prefetchFields;
+
+  public void setField(String field) {
+    this.field = field;
+  }
+
+  public void setPrefetchFields(Set<String> fields) {
+    prefetchFields = fields;
+  }
+
+  @Override
+  public LinkedHashMap<String,Object> paramsToMap() {
+    final LinkedHashMap<String,Object> params = defaultParamsToMap();
+    params.put("field", field);
+    return params;
+  }
+
+  @Override
+  protected void validate() throws FeatureException {
+    if (field == null || field.isEmpty()) {
+      throw new FeatureException(getClass().getSimpleName()+
+          ": field must be provided");
+    }
+  }
+
+  public PrefetchingFieldValueFeature(String name, Map<String,Object> params) {
+    super(name, params);
+  }
+
+  @Override
+  public FeatureWeight createWeight(IndexSearcher searcher, boolean needsScores,
+      SolrQueryRequest request, Query originalQuery, Map<String,String[]> efi)
+          throws IOException {
+    return new PrefetchingFieldValueFeatureWeight(searcher, request, originalQuery, efi);
+  }
+
+  @VisibleForTesting
+  public Set<String> getPrefetchFields(){
+    return prefetchFields;
+  }
+
+  public class PrefetchingFieldValueFeatureWeight extends FieldValueFeatureWeight {
+    private final SchemaField schemaField;
+    private final SolrDocumentFetcher docFetcher;
+
+    public PrefetchingFieldValueFeatureWeight(IndexSearcher searcher,
+        SolrQueryRequest request, Query originalQuery, Map<String,String[]> efi) {
+      super(searcher, request, originalQuery, efi);
+      if (searcher instanceof SolrIndexSearcher) {
+        schemaField = ((SolrIndexSearcher) searcher).getSchema().getFieldOrNull(field);
+      } else {
+        schemaField = null;
+      }
+      this.docFetcher = request.getSearcher().getDocFetcher();
+    }
+
+    /**
+     * Return a FeatureScorer that work with stored fields and makes use of the cache if the configured field is stored
+     * and has no docValues.
+     * Otherwise, delegate the work to the FieldValueFeature.
+     *
+     * @param context the segment this FeatureScorer is working with
+     * @return FeatureScorer for the current segment and field
+     * @throws IOException as defined by abstract class Feature
+     */
+    @Override
+    public FeatureScorer scorer(LeafReaderContext context) throws IOException {
+      if (schemaField != null && !schemaField.stored() && schemaField.hasDocValues()) {
+        return super.scorer(context);
+      }
+      return new PrefetchingFieldValueFeatureScorer(this, context,
+          DocIdSetIterator.all(DocIdSetIterator.NO_MORE_DOCS));
+    }
+
+    /**
+     * A FeatureScorer that reads the stored value for a field
+     * docFetcher does not request a single field but all the prefetchFields to improve performance through caching
+     */
+    public class PrefetchingFieldValueFeatureScorer extends FieldValueFeatureScorer {
+
+      public PrefetchingFieldValueFeatureScorer(FeatureWeight weight,
+          LeafReaderContext context, DocIdSetIterator itr) {
+        super(weight, context, itr);
+      }
+
+      @Override
+      public float score() throws IOException {
+        try {
+          final Document document = docFetcher.doc(context.docBase + itr.docID(), prefetchFields);
+
+          return super.parseStoredFieldValue(document.getField(field));
+        } catch (final IOException e) {
+          throw new FeatureException(
+              e.toString() + ": " +
+                  "Unable to extract feature for "
+                  + name, e);

Review comment:
       For `FieldValueFeature[Scorer]` it is possible to map a feature name to the field whilst troubleshooting if this exception were to be encountered. For `PrefetchingFieldValueFeature[Scorer]` the scenario is more complex though i.e. the root cause of the exception might not be the field of the named feature but another field or fields in the `prefetchFields` set.
   
   Would you have any thoughts on how we might account for that in the exception wording? My initial thought was to include all the `prefetchFields` fields in the exception but on second thought that could be overwhelming if there are quite a few fields in some use cases.
   
   Also with troubleshooting in mind, I wonder if `prefetchFields` being not just a set but an ordered set might be useful (_assuming_ that ordering would _not_ have a runtime performance cost).
   
   Oh and one more thing vaguely related also, overriding `toString(String)` or `toString()` might be something to consider, the super class methods will correctly include the field itself but it will not include the `prefetchFields` since that is specific to this class and not part of the configuration parameters.

##########
File path: solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedFeatureStore.java
##########
@@ -138,15 +147,36 @@ public Object applyUpdatesToManagedData(Object updates) {
       Map<String,Object> updatesMap = (Map<String,Object>) updates;
       final String featureStore = (String) updatesMap.get(FEATURE_STORE_NAME_KEY);
       addFeature(updatesMap, featureStore);
+      updatedFeatureStores.add(featureStore);
     }
 
+    preparePrefetchingFieldValueFeatures(updatedFeatureStores);
+
     final List<Object> features = new ArrayList<>();
     for (final FeatureStore fs : stores.values()) {
       features.addAll(featuresAsManagedResources(fs));
     }
     return features;
   }
 
+  private void preparePrefetchingFieldValueFeatures(Set<String> updatedFeatureStores) {
+    for(String featureStoreName: updatedFeatureStores) {
+      final Set<PrefetchingFieldValueFeature> prefetchingFieldValueFeatures = getFeatureStore(featureStoreName)
+          .getFeatures().stream()
+          .filter(feature -> feature instanceof PrefetchingFieldValueFeature)
+          .map(feature -> ((PrefetchingFieldValueFeature) feature))
+          .collect(Collectors.toSet());

Review comment:
       Very elegant! :-)

##########
File path: solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/PrefetchingFieldValueFeature.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.SolrDocumentFetcher;
+import org.apache.solr.search.SolrIndexSearcher;
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This feature returns the value of a field in the current document.
+ * The field must have stored="true" or docValues="true" properties.
+ * Example configuration:
+ * <pre>{
+  "name":  "rawHits",
+  "class": "org.apache.solr.ltr.feature.FieldValueFeature",
+  "params": {
+      "field": "hits"
+  }
+}</pre>
+ */
+public class PrefetchingFieldValueFeature extends FieldValueFeature {
+  // used to store all fields from all PrefetchingFieldValueFeatures
+  private Set<String> prefetchFields;
+
+  public void setField(String field) {
+    this.field = field;
+  }
+
+  public void setPrefetchFields(Set<String> fields) {
+    prefetchFields = fields;
+  }
+
+  @Override
+  public LinkedHashMap<String,Object> paramsToMap() {
+    final LinkedHashMap<String,Object> params = defaultParamsToMap();
+    params.put("field", field);
+    return params;
+  }
+
+  @Override
+  protected void validate() throws FeatureException {
+    if (field == null || field.isEmpty()) {
+      throw new FeatureException(getClass().getSimpleName()+
+          ": field must be provided");
+    }
+  }
+
+  public PrefetchingFieldValueFeature(String name, Map<String,Object> params) {
+    super(name, params);
+  }
+
+  @Override
+  public FeatureWeight createWeight(IndexSearcher searcher, boolean needsScores,
+      SolrQueryRequest request, Query originalQuery, Map<String,String[]> efi)
+          throws IOException {
+    return new PrefetchingFieldValueFeatureWeight(searcher, request, originalQuery, efi);
+  }
+
+  @VisibleForTesting
+  public Set<String> getPrefetchFields(){
+    return prefetchFields;
+  }
+
+  public class PrefetchingFieldValueFeatureWeight extends FieldValueFeatureWeight {
+    private final SchemaField schemaField;
+    private final SolrDocumentFetcher docFetcher;
+
+    public PrefetchingFieldValueFeatureWeight(IndexSearcher searcher,
+        SolrQueryRequest request, Query originalQuery, Map<String,String[]> efi) {
+      super(searcher, request, originalQuery, efi);
+      if (searcher instanceof SolrIndexSearcher) {
+        schemaField = ((SolrIndexSearcher) searcher).getSchema().getFieldOrNull(field);
+      } else {
+        schemaField = null;
+      }

Review comment:
       ```suggestion
   ```
   
   in favour of super class' `schemaField` becoming `protected` instead of `private`

##########
File path: solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/PrefetchingFieldValueFeature.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.SolrDocumentFetcher;
+import org.apache.solr.search.SolrIndexSearcher;
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This feature returns the value of a field in the current document.
+ * The field must have stored="true" or docValues="true" properties.
+ * Example configuration:
+ * <pre>{
+  "name":  "rawHits",
+  "class": "org.apache.solr.ltr.feature.FieldValueFeature",
+  "params": {
+      "field": "hits"
+  }
+}</pre>
+ */
+public class PrefetchingFieldValueFeature extends FieldValueFeature {
+  // used to store all fields from all PrefetchingFieldValueFeatures
+  private Set<String> prefetchFields;
+
+  public void setField(String field) {
+    this.field = field;
+  }

Review comment:
       ```suggestion
   ```
   
   Use of the super class' `setField` should work (I think, or if not let's add comment w.r.t. why we override) and whilst at first glance it seems that the super class also filling the singleton`fieldAsSet` could be marginally wasteful I suspect it might actually be necessary if this class here were to end up delegating to the super class implementation where the super class implementation might use `fieldAsSet`. Also less code here if we don't have a setField override.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org