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 2017/12/01 15:43:36 UTC

[1/2] lucene-solr:branch_7x: SOLR-11250: A new DefaultWrapperModel class for loading of large and/or externally stored LTRScoringModel definitions. (Yuki Yano, shalin, Christine Poerschke)

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x d2554218c -> a8fbff4d1


SOLR-11250: A new DefaultWrapperModel class for loading of large and/or externally stored LTRScoringModel definitions. (Yuki Yano, shalin, Christine Poerschke)


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

Branch: refs/heads/branch_7x
Commit: 1cbe4db460fe4bff2364419e3d9c83fade78ed9c
Parents: d255421
Author: Christine Poerschke <cp...@apache.org>
Authored: Tue Nov 28 14:55:57 2017 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Fri Dec 1 15:26:46 2017 +0000

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 solr/contrib/ltr/build.xml                      |   5 +
 solr/contrib/ltr/ivy.xml                        |   5 +-
 .../solr/ltr/model/DefaultWrapperModel.java     | 105 +++++++
 .../org/apache/solr/ltr/model/WrapperModel.java | 169 +++++++++++
 .../solr/ltr/store/rest/ManagedModelStore.java  |  35 ++-
 .../solr/collection1/conf/solrconfig-ltr.xml    |   3 +
 .../solr/ltr/model/TestDefaultWrapperModel.java | 145 ++++++++++
 .../apache/solr/ltr/model/TestWrapperModel.java | 290 +++++++++++++++++++
 .../store/rest/TestModelManagerPersistence.java |  76 +++++
 solr/solr-ref-guide/src/learning-to-rank.adoc   |  49 ++++
 11 files changed, 878 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1cbe4db4/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 86d81ad..698f02a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -81,6 +81,9 @@ New Features
 
 * SOLR-11202: Implement a set-property command for AutoScaling API. (ab, shalin)
 
+* SOLR-11250: A new DefaultWrapperModel class for loading of large and/or externally stored
+  LTRScoringModel definitions. (Yuki Yano, shalin, Christine Poerschke)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1cbe4db4/solr/contrib/ltr/build.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/build.xml b/solr/contrib/ltr/build.xml
index bbd5cf3..a5778c4 100644
--- a/solr/contrib/ltr/build.xml
+++ b/solr/contrib/ltr/build.xml
@@ -25,6 +25,11 @@
 
   <import file="../contrib-build.xml"/>
 
+  <path id="test.classpath">
+    <path refid="solr.test.base.classpath"/>
+    <fileset dir="${test.lib.dir}" includes="*.jar"/>
+  </path>
+
   <target name="compile-core" depends=" solr-contrib-build.compile-core"/>
 
 </project>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1cbe4db4/solr/contrib/ltr/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/ivy.xml b/solr/contrib/ltr/ivy.xml
index 68e9797..3b7e1c7 100644
--- a/solr/contrib/ltr/ivy.xml
+++ b/solr/contrib/ltr/ivy.xml
@@ -24,9 +24,10 @@
     </configurations>
 
    <dependencies>
-
-
      <dependency org="org.slf4j" name="jcl-over-slf4j" rev="${/org.slf4j/jcl-over-slf4j}" conf="test"/>
+     <dependency org="org.mockito" name="mockito-core" rev="${/org.mockito/mockito-core}" conf="test"/>
+     <dependency org="net.bytebuddy" name="byte-buddy" rev="${/net.bytebuddy/byte-buddy}" conf="test"/>
+     <dependency org="org.objenesis" name="objenesis" rev="${/org.objenesis/objenesis}" conf="test"/>
      <exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}"/>
    </dependencies>
 </ivy-module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1cbe4db4/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/DefaultWrapperModel.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/DefaultWrapperModel.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/DefaultWrapperModel.java
new file mode 100644
index 0000000..b21b6c3
--- /dev/null
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/DefaultWrapperModel.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.ltr.model;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.ltr.feature.Feature;
+import org.apache.solr.ltr.norm.Normalizer;
+import org.noggit.JSONParser;
+import org.noggit.ObjectBuilder;
+
+/**
+ * A scoring model that fetches the wrapped model from {@link SolrResourceLoader}.
+ *
+ * <p>This model uses {@link SolrResourceLoader#openResource(String)} for fetching the wrapped model.
+ * If you give a relative path for {@code params/resource}, this model will try to load the wrapped model from
+ * the instance directory (i.e. ${solr.solr.home}). Otherwise, seek through classpaths.
+ *
+ * <p>Example configuration:
+ * <pre>{
+  "class": "org.apache.solr.ltr.model.DefaultWrapperModel",
+  "name": "myWrapperModelName",
+  "params": {
+    "resource": "models/myModel.json"
+  }
+}</pre>
+ *
+ * @see SolrResourceLoader#openResource(String)
+ */
+public class DefaultWrapperModel extends WrapperModel {
+
+  private String resource;
+
+  public DefaultWrapperModel(String name, List<Feature> features, List<Normalizer> norms, String featureStoreName,
+      List<Feature> allFeatures, Map<String, Object> params) {
+    super(name, features, norms, featureStoreName, allFeatures, params);
+  }
+
+  public void setResource(String resource) {
+    this.resource = resource;
+  }
+
+  @Override
+  protected void validate() throws ModelException {
+    super.validate();
+    if (resource == null) {
+      throw new ModelException("no resource configured for model "+name);
+    }
+  }
+
+  @Override
+  public Map<String, Object> fetchModelMap() throws ModelException {
+    Map<String, Object> modelMapObj;
+    try (InputStream in = openInputStream()) {
+      modelMapObj = parseInputStream(in);
+    } catch (IOException e) {
+      throw new ModelException("Failed to fetch the wrapper model from given resource (" + resource + ")", e);
+    }
+    return modelMapObj;
+  }
+
+  protected InputStream openInputStream() throws IOException {
+    return solrResourceLoader.openResource(resource);
+  }
+
+  @SuppressWarnings("unchecked")
+  protected Map<String, Object> parseInputStream(InputStream in) throws IOException {
+    try (Reader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) {
+      return (Map<String, Object>) new ObjectBuilder(new JSONParser(reader)).getVal();
+    }
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(getClass().getSimpleName());
+    sb.append("(name=").append(getName());
+    sb.append(",resource=").append(resource);
+    sb.append(",model=(").append(model.toString()).append(")");
+
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1cbe4db4/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/WrapperModel.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/WrapperModel.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/WrapperModel.java
new file mode 100644
index 0000000..cf66135
--- /dev/null
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/WrapperModel.java
@@ -0,0 +1,169 @@
+/*
+ * 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.model;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.Explanation;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.ltr.feature.Feature;
+import org.apache.solr.ltr.norm.Normalizer;
+
+/**
+ * A scoring model that wraps the other model.
+ *
+ * <p>This model loads a model from an external resource during the initialization.
+ * The way of fetching the wrapped model is depended on
+ * the implementation of {@link WrapperModel#fetchModelMap()}.
+ *
+ * <p>This model doesn't hold the actual parameters of the wrapped model,
+ * thus it can manage large models which are difficult to upload to ZooKeeper.
+ *
+ * <p>Example configuration:
+ * <pre>{
+    "class": "...",
+    "name": "myModelName",
+    "params": {
+        ...
+    }
+ }</pre>
+ *
+ * <p>NOTE: no "features" are configured in the wrapper model
+ * because the wrapped model's features will be used instead.
+ * Also note that if a "store" is configured for the wrapper
+ * model then it must match the "store" of the wrapped model.
+ */
+public abstract class WrapperModel extends LTRScoringModel {
+
+  protected SolrResourceLoader solrResourceLoader;
+  protected LTRScoringModel model;
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = super.hashCode();
+    result = prime * result + ((model == null) ? 0 : model.hashCode());
+    result = prime * result + ((solrResourceLoader == null) ? 0 : solrResourceLoader.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) return true;
+    if (!super.equals(obj)) return false;
+    if (getClass() != obj.getClass()) return false;
+    WrapperModel other = (WrapperModel) obj;
+    if (model == null) {
+      if (other.model != null) return false;
+    } else if (!model.equals(other.model)) return false;
+    if (solrResourceLoader == null) {
+      if (other.solrResourceLoader != null) return false;
+    } else if (!solrResourceLoader.equals(other.solrResourceLoader)) return false;
+    return true;
+  }
+
+  public WrapperModel(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
+  protected void validate() throws ModelException {
+    if (!features.isEmpty()) {
+      throw new ModelException("features must be empty for the wrapper model " + name);
+    }
+    if (!norms.isEmpty()) {
+      throw new ModelException("norms must be empty for the wrapper model " + name);
+    }
+
+    if (model != null) {
+      super.validate();
+      model.validate();
+      // check feature store names match
+      final String wrappedFeatureStoreName = model.getFeatureStoreName();
+      if (wrappedFeatureStoreName == null || !wrappedFeatureStoreName.equals(this.getFeatureStoreName())) {
+        throw new ModelException("wrapper feature store name ("+this.getFeatureStoreName() +")"
+            + " must match the "
+            + "wrapped feature store name ("+wrappedFeatureStoreName+")");
+      }
+    }
+  }
+
+  public void setSolrResourceLoader(SolrResourceLoader solrResourceLoader) {
+    this.solrResourceLoader = solrResourceLoader;
+  }
+
+  public void updateModel(LTRScoringModel model) {
+    this.model = model;
+    validate();
+  }
+
+  /*
+   * The child classes must implement how to fetch the definition of the wrapped model.
+   */
+  public abstract Map<String, Object> fetchModelMap() throws ModelException;
+
+  @Override
+  public List<Normalizer> getNorms() {
+    return model.getNorms();
+  }
+
+  @Override
+  public List<Feature> getFeatures() {
+    return model.getFeatures();
+  }
+
+  @Override
+  public Collection<Feature> getAllFeatures() {
+    return model.getAllFeatures();
+  }
+
+  @Override
+  public float score(float[] modelFeatureValuesNormalized) {
+    return model.score(modelFeatureValuesNormalized);
+  }
+
+  @Override
+  public Explanation explain(LeafReaderContext context, int doc, float finalScore,
+                             List<Explanation> featureExplanations) {
+    return model.explain(context, doc, finalScore, featureExplanations);
+  }
+
+  @Override
+  public void normalizeFeaturesInPlace(float[] modelFeatureValues) {
+    model.normalizeFeaturesInPlace(modelFeatureValues);
+  }
+
+  @Override
+  public Explanation getNormalizerExplanation(Explanation e, int idx) {
+    return model.getNormalizerExplanation(e, idx);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(getClass().getSimpleName());
+    sb.append("(name=").append(getName());
+    sb.append(",model=(").append(model.toString()).append(")");
+
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1cbe4db4/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedModelStore.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedModelStore.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedModelStore.java
index 79640c1..342a140 100644
--- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedModelStore.java
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedModelStore.java
@@ -27,6 +27,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.ltr.feature.Feature;
+import org.apache.solr.ltr.model.WrapperModel;
 import org.apache.solr.ltr.model.LTRScoringModel;
 import org.apache.solr.ltr.model.ModelException;
 import org.apache.solr.ltr.norm.IdentityNormalizer;
@@ -231,7 +232,7 @@ public class ManagedModelStore extends ManagedResource implements ManagedResourc
       }
     }
 
-    return LTRScoringModel.getInstance(solrResourceLoader,
+    final LTRScoringModel ltrScoringModel = LTRScoringModel.getInstance(solrResourceLoader,
         (String) modelMap.get(CLASS_KEY), // modelClassName
         (String) modelMap.get(NAME_KEY), // modelName
         features,
@@ -239,6 +240,28 @@ public class ManagedModelStore extends ManagedResource implements ManagedResourc
         featureStore.getName(),
         featureStore.getFeatures(),
         (Map<String,Object>) modelMap.get(PARAMS_KEY));
+
+    if (ltrScoringModel instanceof WrapperModel) {
+      initWrapperModel(solrResourceLoader, (WrapperModel)ltrScoringModel, managedFeatureStore);
+    }
+
+    return ltrScoringModel;
+  }
+
+  private static void initWrapperModel(SolrResourceLoader solrResourceLoader,
+                                       WrapperModel wrapperModel, ManagedFeatureStore managedFeatureStore) {
+    wrapperModel.setSolrResourceLoader(solrResourceLoader);
+    final LTRScoringModel model = fromLTRScoringModelMap(
+        solrResourceLoader,
+        wrapperModel.fetchModelMap(),
+        managedFeatureStore);
+    if (model instanceof WrapperModel) {
+      log.warn("It is unusual for one WrapperModel ({}) to wrap another WrapperModel ({})",
+          wrapperModel.getName(),
+          model.getName());
+      initWrapperModel(solrResourceLoader, (WrapperModel)model, managedFeatureStore);
+    }
+    wrapperModel.updateModel(model);
   }
 
   private static LinkedHashMap<String,Object> toLTRScoringModelMap(LTRScoringModel model) {
@@ -249,10 +272,12 @@ public class ManagedModelStore extends ManagedResource implements ManagedResourc
     modelMap.put(STORE_KEY, model.getFeatureStoreName());
 
     final List<Map<String,Object>> features = new ArrayList<>();
-    final List<Feature> featuresList = model.getFeatures();
-    final List<Normalizer> normsList = model.getNorms();
-    for (int ii=0; ii<featuresList.size(); ++ii) {
-      features.add(toFeatureMap(featuresList.get(ii), normsList.get(ii)));
+    if (!(model instanceof WrapperModel)) {
+      final List<Feature> featuresList = model.getFeatures();
+      final List<Normalizer> normsList = model.getNorms();
+      for (int ii = 0; ii < featuresList.size(); ++ii) {
+        features.add(toFeatureMap(featuresList.get(ii), normsList.get(ii)));
+      }
     }
     modelMap.put(FEATURES_KEY, features);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1cbe4db4/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 4f1f5ca..c551938 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
@@ -16,6 +16,9 @@
  <directoryFactory name="DirectoryFactory"
   class="${solr.directoryFactory:solr.RAMDirectoryFactory}" />
 
+ <!-- for use with the DefaultWrapperModel class -->
+ <lib dir="${solr.solr.home:.}/models" />
+
  <schemaFactory class="ClassicIndexSchemaFactory" />
 
  <requestDispatcher>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1cbe4db4/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestDefaultWrapperModel.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestDefaultWrapperModel.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestDefaultWrapperModel.java
new file mode 100644
index 0000000..a930c23
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestDefaultWrapperModel.java
@@ -0,0 +1,145 @@
+/*
+ * 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.model;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.TestRerankBase;
+import org.apache.solr.ltr.feature.Feature;
+import org.apache.solr.ltr.feature.FieldValueFeature;
+import org.apache.solr.ltr.feature.ValueFeature;
+import org.apache.solr.ltr.store.rest.ManagedModelStore;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestDefaultWrapperModel extends TestRerankBase {
+
+  final private static String featureStoreName = "test";
+  private static String baseModelJson = null;
+  private static File baseModelFile = null;
+
+  static List<Feature> features = null;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    setuptest(false);
+    assertU(adoc("id", "1", "title", "w1", "description", "w1", "popularity", "1"));
+    assertU(adoc("id", "2", "title", "w2", "description", "w2", "popularity", "2"));
+    assertU(adoc("id", "3", "title", "w3", "description", "w3", "popularity", "3"));
+    assertU(adoc("id", "4", "title", "w4", "description", "w4", "popularity", "4"));
+    assertU(adoc("id", "5", "title", "w5", "description", "w5", "popularity", "5"));
+    assertU(commit());
+
+    loadFeature("popularity", FieldValueFeature.class.getCanonicalName(), "test", "{\"field\":\"popularity\"}");
+    loadFeature("const", ValueFeature.class.getCanonicalName(), "test", "{\"value\":5}");
+    features = new ArrayList<>();
+    features.add(getManagedFeatureStore().getFeatureStore("test").get("popularity"));
+    features.add(getManagedFeatureStore().getFeatureStore("test").get("const"));
+
+    baseModelJson = getModelInJson("linear", LinearModel.class.getCanonicalName(),
+        new String[] {"popularity", "const"},
+        featureStoreName,
+        "{\"weights\":{\"popularity\":-1.0, \"const\":1.0}}");
+    // prepare the base model as a file resource
+    baseModelFile = new File(tmpConfDir, "baseModel.json");
+    try (BufferedWriter writer = new BufferedWriter(
+        new OutputStreamWriter(new FileOutputStream(baseModelFile), StandardCharsets.UTF_8))) {
+      writer.write(baseModelJson);
+    }
+    baseModelFile.deleteOnExit();
+  }
+
+  private static String getDefaultWrapperModelInJson(String wrapperModelName, String[] features, String params) {
+    return getModelInJson(wrapperModelName, DefaultWrapperModel.class.getCanonicalName(),
+        features, featureStoreName, params);
+  }
+
+  @Test
+  public void testLoadModelFromResource() throws Exception {
+    String wrapperModelJson = getDefaultWrapperModelInJson("fileWrapper",
+        new String[0],
+        "{\"resource\":\"" + baseModelFile.getName() + "\"}");
+    assertJPut(ManagedModelStore.REST_END_POINT, wrapperModelJson, "/responseHeader/status==0");
+
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("{!func}pow(popularity,2)");
+    query.add("rows", "3");
+    query.add("fl", "*,score");
+    query.add("rq", "{!ltr reRankDocs=3 model=fileWrapper}");
+    assertJQ("/query" + query.toQueryString(),
+             "/response/docs/[0]/id==\"3\"", "/response/docs/[0]/score==2.0",
+             "/response/docs/[1]/id==\"4\"", "/response/docs/[1]/score==1.0",
+             "/response/docs/[2]/id==\"5\"", "/response/docs/[2]/score==0.0");
+  }
+
+  @Test
+  public void testLoadNestedWrapperModel() throws Exception {
+    String otherWrapperModelJson = getDefaultWrapperModelInJson("otherNestedWrapper",
+        new String[0],
+        "{\"resource\":\"" + baseModelFile.getName() + "\"}");
+    File otherWrapperModelFile = new File(tmpConfDir, "nestedWrapperModel.json");
+    try (BufferedWriter writer = new BufferedWriter(
+        new OutputStreamWriter(new FileOutputStream(otherWrapperModelFile), StandardCharsets.UTF_8))) {
+      writer.write(otherWrapperModelJson);
+    }
+
+    String wrapperModelJson = getDefaultWrapperModelInJson("nestedWrapper",
+        new String[0],
+        "{\"resource\":\"" + otherWrapperModelFile.getName() + "\"}");
+    assertJPut(ManagedModelStore.REST_END_POINT, wrapperModelJson, "/responseHeader/status==0");
+    final SolrQuery query = new SolrQuery();
+    query.setQuery("{!func}pow(popularity,2)");
+    query.add("rows", "3");
+    query.add("fl", "*,score");
+    query.add("rq", "{!ltr reRankDocs=3 model=nestedWrapper}");
+    assertJQ("/query" + query.toQueryString(),
+             "/response/docs/[0]/id==\"3\"", "/response/docs/[0]/score==2.0",
+             "/response/docs/[1]/id==\"4\"", "/response/docs/[1]/score==1.0",
+             "/response/docs/[2]/id==\"5\"", "/response/docs/[2]/score==0.0");
+  }
+
+  @Test
+  public void testLoadModelFromUnknownResource() throws Exception {
+    String wrapperModelJson = getDefaultWrapperModelInJson("unknownWrapper",
+        new String[0],
+        "{\"resource\":\"unknownModel.json\"}");
+    assertJPut(ManagedModelStore.REST_END_POINT, wrapperModelJson,
+               "/responseHeader/status==400",
+               "/error/msg==\"org.apache.solr.ltr.model.ModelException: "
+               + "Failed to fetch the wrapper model from given resource (unknownModel.json)\"");
+  }
+
+  @Test
+  public void testLoadModelWithEmptyParams() throws Exception {
+    String wrapperModelJson = getDefaultWrapperModelInJson("invalidWrapper",
+        new String[0],
+        "{}");
+    assertJPut(ManagedModelStore.REST_END_POINT, wrapperModelJson,
+               "/responseHeader/status==400",
+               "/error/msg==\"org.apache.solr.ltr.model.ModelException: "
+               + "no resource configured for model invalidWrapper\"");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1cbe4db4/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestWrapperModel.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestWrapperModel.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestWrapperModel.java
new file mode 100644
index 0000000..78818ff
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestWrapperModel.java
@@ -0,0 +1,290 @@
+/*
+ * 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.model;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.ltr.TestRerankBase;
+import org.apache.solr.ltr.feature.Feature;
+import org.apache.solr.ltr.feature.ValueFeature;
+import org.apache.solr.ltr.norm.IdentityNormalizer;
+import org.apache.solr.ltr.norm.Normalizer;
+import org.apache.solr.ltr.store.FeatureStore;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestWrapperModel extends TestRerankBase {
+
+  private static class StubWrapperModel extends WrapperModel {
+
+    private StubWrapperModel(String name) {
+      this(name, Collections.emptyList(), Collections.emptyList());
+    }
+
+    private StubWrapperModel(String name, List<Feature> features, List<Normalizer> norms) {
+      super(name, features, norms, FeatureStore.DEFAULT_FEATURE_STORE_NAME, features, Collections.emptyMap());
+    }
+
+    @Override
+    public Map<String, Object> fetchModelMap() throws ModelException {
+      return null;
+    }
+  }
+
+  private static LTRScoringModel createMockWrappedModel(String featureStoreName,
+      List<Feature> features, List<Normalizer> norms) {
+      LTRScoringModel wrappedModel = Mockito.mock(LTRScoringModel.class);
+      Mockito.doReturn(featureStoreName).when(wrappedModel).getFeatureStoreName();
+      Mockito.doReturn(features).when(wrappedModel).getFeatures();
+      Mockito.doReturn(norms).when(wrappedModel).getNorms();
+      return wrappedModel;
+  }
+
+  @Test
+  public void testValidate() throws Exception {
+    WrapperModel wrapperModel = new StubWrapperModel("testModel");
+    try {
+      wrapperModel.validate();
+    } catch (ModelException e) {
+      fail("Validation must succeed if no wrapped model is set");
+    }
+
+    // wrapper model with features
+    WrapperModel wrapperModelWithFeatures = new StubWrapperModel("testModel",
+        Collections.singletonList(new ValueFeature("val", Collections.emptyMap())), Collections.emptyList());
+    try {
+      wrapperModelWithFeatures.validate();
+      fail("Validation must fail if features of the wrapper model isn't empty");
+    } catch (ModelException e) {
+      assertEquals("features must be empty for the wrapper model testModel", e.getMessage());
+    }
+
+    // wrapper model with norms
+    WrapperModel wrapperModelWithNorms = new StubWrapperModel("testModel",
+        Collections.emptyList(), Collections.singletonList(IdentityNormalizer.INSTANCE));
+    try {
+      wrapperModelWithNorms.validate();
+      fail("Validation must fail if norms of the wrapper model isn't empty");
+    } catch (ModelException e) {
+      assertEquals("norms must be empty for the wrapper model testModel", e.getMessage());
+    }
+
+    assumeWorkingMockito();
+
+    // update valid model
+    {
+      LTRScoringModel wrappedModel = 
+          createMockWrappedModel(FeatureStore.DEFAULT_FEATURE_STORE_NAME,
+              Arrays.asList(
+                  new ValueFeature("v1", Collections.emptyMap()),
+                  new ValueFeature("v2", Collections.emptyMap())),
+              Arrays.asList(
+                  IdentityNormalizer.INSTANCE,
+                  IdentityNormalizer.INSTANCE)
+              );
+      try {
+        wrapperModel.updateModel(wrappedModel);
+      } catch (ModelException e) {
+        fail("Validation must succeed if the wrapped model is valid");
+      }
+    }
+
+    // update invalid model (feature store mismatch)
+    {
+      LTRScoringModel wrappedModel = 
+          createMockWrappedModel("wrappedFeatureStore",
+              Arrays.asList(
+                  new ValueFeature("v1", Collections.emptyMap()),
+                  new ValueFeature("v2", Collections.emptyMap())),
+              Arrays.asList(
+                  IdentityNormalizer.INSTANCE,
+                  IdentityNormalizer.INSTANCE)
+              );
+      try {
+        wrapperModel.updateModel(wrappedModel);
+        fail("Validation must fail if wrapped model feature store differs from wrapper model feature store");
+      } catch (ModelException e) {
+        assertEquals("wrapper feature store name (_DEFAULT_) must match the wrapped feature store name (wrappedFeatureStore)", e.getMessage());
+      }
+    }
+
+    // update invalid model (no features)
+    {
+      LTRScoringModel wrappedModel = 
+          createMockWrappedModel(FeatureStore.DEFAULT_FEATURE_STORE_NAME,
+              Collections.emptyList(),
+              Arrays.asList(
+                  IdentityNormalizer.INSTANCE,
+                  IdentityNormalizer.INSTANCE)
+              );
+      try {
+        wrapperModel.updateModel(wrappedModel);
+        fail("Validation must fail if the wrapped model is invalid");
+      } catch (ModelException e) {
+        assertEquals("no features declared for model testModel", e.getMessage());
+      }
+    }
+
+    // update invalid model (no norms)
+    {
+      LTRScoringModel wrappedModel = 
+          createMockWrappedModel(FeatureStore.DEFAULT_FEATURE_STORE_NAME,
+              Arrays.asList(
+                  new ValueFeature("v1", Collections.emptyMap()),
+                  new ValueFeature("v2", Collections.emptyMap())),
+              Collections.emptyList()
+              );
+      try {
+        wrapperModel.updateModel(wrappedModel);
+        fail("Validation must fail if the wrapped model is invalid");
+      } catch (ModelException e) {
+        assertEquals("counted 2 features and 0 norms in model testModel", e.getMessage());
+      }
+    }
+  }
+
+  @Test
+  public void testMethodOverridesAndDelegation() throws Exception {
+    assumeWorkingMockito();
+    final int overridableMethodCount = testOverwrittenMethods();
+    final int methodCount = testDelegateMethods();
+    assertEquals("method count mismatch", overridableMethodCount, methodCount);
+  }
+
+  private int testOverwrittenMethods() throws Exception {
+    int overridableMethodCount = 0;
+    for (final Method superClassMethod : LTRScoringModel.class.getDeclaredMethods()) {
+      final int modifiers = superClassMethod.getModifiers();
+      if (Modifier.isFinal(modifiers)) continue;
+      if (Modifier.isStatic(modifiers)) continue;
+
+      ++overridableMethodCount;
+      if (Arrays.asList(
+          "getName",  // the wrapper model's name is its own name i.e. _not_ the name of the wrapped model
+          "getFeatureStoreName", // wrapper and wrapped model feature store should match, so need not override
+          "getParams" // the wrapper model's params are its own params i.e. _not_ the params of the wrapped model
+          ).contains(superClassMethod.getName())) {
+        try {
+          final Method subClassMethod = WrapperModel.class.getDeclaredMethod(
+              superClassMethod.getName(),
+              superClassMethod.getParameterTypes());
+          fail(WrapperModel.class + " need not override\n'" + superClassMethod + "'"
+               + " but it does override\n'" + subClassMethod + "'");
+        } catch (NoSuchMethodException e) {
+          // ok
+        }
+      } else {
+        try {
+          final Method subClassMethod = WrapperModel.class.getDeclaredMethod(
+              superClassMethod.getName(),
+              superClassMethod.getParameterTypes());
+          assertEquals("getReturnType() difference",
+              superClassMethod.getReturnType(),
+              subClassMethod.getReturnType());
+        } catch (NoSuchMethodException e) {
+          fail(WrapperModel.class + " needs to override '" + superClassMethod + "'");
+        }
+      }
+    }
+    return overridableMethodCount;
+  }
+
+  private int testDelegateMethods() throws Exception {
+    int methodCount = 0;
+    WrapperModel wrapperModel = Mockito.spy(new StubWrapperModel("testModel"));
+
+    // ignore validate in this test case
+    Mockito.doNothing().when(wrapperModel).validate();
+    ++methodCount;
+
+    LTRScoringModel wrappedModel = Mockito.mock(LTRScoringModel.class);
+    wrapperModel.updateModel(wrappedModel);
+
+    // cannot be stubbed or verified
+    ++methodCount; // toString
+    ++methodCount; // hashCode
+    ++methodCount; // equals
+
+    // getFeatureStoreName : not delegate
+    Mockito.reset(wrappedModel);
+    wrapperModel.getFeatureStoreName();
+    ++methodCount;
+    Mockito.verify(wrappedModel, Mockito.times(0)).getFeatureStoreName();
+
+    // getName : not delegate
+    Mockito.reset(wrappedModel);
+    wrapperModel.getName();
+    ++methodCount;
+    Mockito.verify(wrappedModel, Mockito.times(0)).getName();
+
+    // getParams : not delegate
+    Mockito.reset(wrappedModel);
+    wrapperModel.getParams();
+    ++methodCount;
+    Mockito.verify(wrappedModel, Mockito.times(0)).getParams();
+
+    // getNorms : delegate
+    Mockito.reset(wrappedModel);
+    wrapperModel.getNorms();
+    ++methodCount;
+    Mockito.verify(wrappedModel, Mockito.times(1)).getNorms();
+
+    // getFeatures : delegate
+    Mockito.reset(wrappedModel);
+    wrapperModel.getFeatures();
+    ++methodCount;
+    Mockito.verify(wrappedModel, Mockito.times(1)).getFeatures();
+
+    // getAllFeatures : delegate
+    Mockito.reset(wrappedModel);
+    wrapperModel.getAllFeatures();
+    ++methodCount;
+    Mockito.verify(wrappedModel, Mockito.times(1)).getAllFeatures();
+
+    // score : delegate
+    Mockito.reset(wrappedModel);
+    wrapperModel.score(null);
+    ++methodCount;
+    Mockito.verify(wrappedModel, Mockito.times(1)).score(null);
+
+    // normalizeFeaturesInPlace : delegate
+    Mockito.reset(wrappedModel);
+    wrapperModel.normalizeFeaturesInPlace(null);
+    ++methodCount;
+    Mockito.verify(wrappedModel, Mockito.times(1)).normalizeFeaturesInPlace(null);
+
+    // getNormalizerExplanation : delegate
+    Mockito.reset(wrappedModel);
+    wrapperModel.getNormalizerExplanation(null, 0);
+    ++methodCount;
+    Mockito.verify(wrappedModel, Mockito.times(1)).getNormalizerExplanation(null, 0);
+
+    // explain : delegate
+    Mockito.reset(wrappedModel);
+    wrapperModel.explain(null, 0, 0.0f, null);
+    ++methodCount;
+    Mockito.verify(wrappedModel, Mockito.times(1)).explain(null, 0, 0.0f, null);
+
+    return methodCount;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1cbe4db4/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java
index 9dc28e6..a056cf7 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java
@@ -16,12 +16,19 @@
  */
 package org.apache.solr.ltr.store.rest;
 
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Map;
 
 import org.apache.commons.io.FileUtils;
 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.DefaultWrapperModel;
 import org.apache.solr.ltr.model.LinearModel;
 import org.apache.solr.ltr.store.FeatureStore;
 import org.junit.BeforeClass;
@@ -185,4 +192,73 @@ public class TestModelManagerPersistence extends TestRerankBase {
     assertJQ(ManagedFeatureStore.REST_END_POINT + "/" + FeatureStore.DEFAULT_FEATURE_STORE_NAME,
         "/features/==[]");
   }
+
+  private static void doWrapperModelPersistenceChecks(String modelName,
+      String featureStoreName, String baseModelFileName) throws Exception {
+    // note that the wrapper and the wrapped model always have the same name
+    assertJQ(ManagedModelStore.REST_END_POINT,
+        // the wrapped model shouldn't be registered
+        "!/models/[1]/name=='"+modelName+"'",
+        // but the wrapper model should be registered
+        "/models/[0]/name=='"+modelName+"'",
+        "/models/[0]/class=='" + DefaultWrapperModel.class.getCanonicalName() + "'",
+        "/models/[0]/store=='" + featureStoreName + "'",
+        // the wrapper model shouldn't contain the definitions of the wrapped model
+        "/models/[0]/features/==[]",
+        // but only its own parameters
+        "/models/[0]/params=={resource:'"+baseModelFileName+"'}");
+  }
+
+  @Test
+  public void testWrapperModelPersistence() throws Exception {
+    final String modelName = "linear";
+    final String FS_NAME = "testWrapper";
+
+    // check whether models and features are empty
+    assertJQ(ManagedModelStore.REST_END_POINT,
+             "/models/==[]");
+    assertJQ(ManagedFeatureStore.REST_END_POINT + "/" + FS_NAME,
+             "/features/==[]");
+
+    // setup features
+    loadFeature("popularity", FieldValueFeature.class.getCanonicalName(), FS_NAME, "{\"field\":\"popularity\"}");
+    loadFeature("const", ValueFeature.class.getCanonicalName(), FS_NAME, "{\"value\":5}");
+
+    // setup base model
+    String baseModelJson = getModelInJson(modelName, LinearModel.class.getCanonicalName(),
+                                          new String[] {"popularity", "const"}, FS_NAME,
+                                          "{\"weights\":{\"popularity\":-1.0, \"const\":1.0}}");
+    File baseModelFile = new File(tmpConfDir, "baseModelForPersistence.json");
+    try (BufferedWriter writer = new BufferedWriter(
+        new OutputStreamWriter(new FileOutputStream(baseModelFile), StandardCharsets.UTF_8))) {
+      writer.write(baseModelJson);
+    }
+    baseModelFile.deleteOnExit();
+
+    // setup wrapper model
+    String wrapperModelJson = getModelInJson(modelName, DefaultWrapperModel.class.getCanonicalName(),
+                                             new String[0], FS_NAME,
+                                             "{\"resource\":\"" + baseModelFile.getName() + "\"}");
+    assertJPut(ManagedModelStore.REST_END_POINT, wrapperModelJson, "/responseHeader/status==0");
+    doWrapperModelPersistenceChecks(modelName, FS_NAME, baseModelFile.getName());
+
+    // check persistence after reload
+    restTestHarness.reload();
+    doWrapperModelPersistenceChecks(modelName, FS_NAME, baseModelFile.getName());
+
+    // check persistence after restart
+    jetty.stop();
+    jetty.start();
+    doWrapperModelPersistenceChecks(modelName, FS_NAME, baseModelFile.getName());
+
+    // delete test settings
+    restTestHarness.delete(ManagedModelStore.REST_END_POINT + "/" + modelName);
+    restTestHarness.delete(ManagedFeatureStore.REST_END_POINT + "/" + FS_NAME);
+    assertJQ(ManagedModelStore.REST_END_POINT,
+             "/models/==[]");
+    assertJQ(ManagedFeatureStore.REST_END_POINT + "/" + FS_NAME,
+             "/features/==[]");
+
+    // NOTE: we don't test the persistence of the deletion here because it's tested in testFilePersistence
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1cbe4db4/solr/solr-ref-guide/src/learning-to-rank.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/learning-to-rank.adoc b/solr/solr-ref-guide/src/learning-to-rank.adoc
index 02475c6..3bbc34d 100644
--- a/solr/solr-ref-guide/src/learning-to-rank.adoc
+++ b/solr/solr-ref-guide/src/learning-to-rank.adoc
@@ -87,6 +87,7 @@ Feature selection and model training take place offline and outside Solr. The lt
 |General form |Class |Specific examples
 |Linear |{solr-javadocs}/solr-ltr/org/apache/solr/ltr/model/LinearModel.html[LinearModel] |RankSVM, Pranking
 |Multiple Additive Trees |{solr-javadocs}/solr-ltr/org/apache/solr/ltr/model/MultipleAdditiveTreesModel.html[MultipleAdditiveTreesModel] |LambdaMART, Gradient Boosted Regression Trees (GBRT)
+|(wrapper) |{solr-javadocs}/solr-ltr/org/apache/solr/ltr/model/DefaultWrapperModel.html[DefaultWrapperModel] |(not applicable)
 |(custom) |(custom class extending {solr-javadocs}/solr-ltr/org/apache/solr/ltr/model/LTRScoringModel.html[LTRScoringModel]) |(not applicable)
 |===
 
@@ -509,6 +510,54 @@ To delete the `currentFeatureStore` feature store:
 curl -XDELETE 'http://localhost:8983/solr/techproducts/schema/feature-store/currentFeatureStore'
 ----
 
+==== Using large models
+
+With SolrCloud, large models may fail to upload due to the limitation of ZooKeeper's buffer. In this case, `DefaultWrapperModel` may help you to separate the model definition from uploaded file.
+
+Assuming that you consider to use a large model placed at `/path/to/models/myModel.json` through `DefaultWrapperModel`.
+
+[source,json]
+----
+{
+  "store" : "largeModelsFeatureStore",
+  "name" : "myModel",
+  "class" : ...,
+  "features" : [
+    ...
+  ],
+  "params" : {
+    ...
+  }
+}
+----
+
+First, add the directory to Solr's resource paths by <<lib-directives-in-solrconfig.adoc#lib-directives-in-solrconfig,Lib Directives>>:
+
+[source,xml]
+----
+  <lib dir="/path/to" regex="models" />
+----
+
+Then, configure `DefaultWrapperModel` to wrap `myModel.json`:
+
+[source,json]
+----
+{
+  "store" : "largeModelsFeatureStore",
+  "name" : "myWrapperModel",
+  "class" : "org.apache.solr.ltr.model.DefaultWrapperModel",
+  "params" : {
+    "resource" : "myModel.json"
+  }
+}
+----
+
+`myModel.json` will be loaded during the initialization and be able to use by specifying `model=myWrapperModel`.
+
+NOTE: No `"features"` are configured in `myWrapperModel` because the features of the wrapped model (`myModel`) will be used; also note that the `"store"` configured for the wrapper model must match that of the wrapped model i.e. in this example the feature store called `largeModelsFeatureStore` is used.
+
+CAUTION: `<lib dir="/path/to/models" regex=".*\.json" />` doesn't work as expected in this case, because `SolrResourceLoader` considers given resources as JAR if `<lib />` indicates files.
+
 === Applying Changes
 
 The feature store and the model store are both <<managed-resources.adoc#managed-resources,Managed Resources>>. Changes made to managed resources are not applied to the active Solr components until the Solr collection (or Solr core in single server mode) is reloaded.


[2/2] lucene-solr:branch_7x: SOLR-11291: Factor out abstract metrics/SolrCore[Container]Reporter classes. (Omar Abdelnabi, Christine Poerschke)

Posted by cp...@apache.org.
SOLR-11291: Factor out abstract metrics/SolrCore[Container]Reporter classes. (Omar Abdelnabi, Christine Poerschke)


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

Branch: refs/heads/branch_7x
Commit: a8fbff4d1b8aef343b87a78d0be0fb711a46e53b
Parents: 1cbe4db
Author: Christine Poerschke <cp...@apache.org>
Authored: Tue Nov 28 15:39:55 2017 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Fri Dec 1 15:27:06 2017 +0000

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 ++
 .../solr/metrics/SolrCoreContainerReporter.java | 47 ++++++++++++++++++++
 .../apache/solr/metrics/SolrCoreReporter.java   | 47 ++++++++++++++++++++
 .../apache/solr/metrics/SolrMetricManager.java  | 10 ++---
 .../reporters/solr/SolrClusterReporter.java     | 12 ++---
 .../reporters/solr/SolrShardReporter.java       | 12 ++---
 .../reporters/solr/SolrCloudReportersTest.java  |  8 ++++
 7 files changed, 117 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a8fbff4d/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 698f02a..86ea34e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -196,6 +196,9 @@ Other Changes
 * SOLR-11507: SOLR-11638: Randomize SolrTestCaseJ4.CloudSolrClientBuilder more, and simplify it.
   (Jason Gerlowski, David Smiley)
 
+* SOLR-11291: Factor out abstract metrics/SolrCore[Container]Reporter classes.
+  (Omar Abdelnabi, Christine Poerschke)
+
 ==================  7.1.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a8fbff4d/solr/core/src/java/org/apache/solr/metrics/SolrCoreContainerReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrCoreContainerReporter.java b/solr/core/src/java/org/apache/solr/metrics/SolrCoreContainerReporter.java
new file mode 100644
index 0000000..b612b2e
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrCoreContainerReporter.java
@@ -0,0 +1,47 @@
+/*
+ * 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.metrics;
+
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.PluginInfo;
+
+/**
+ * A {@link SolrMetricReporter} that has access to its {@link CoreContainer}.
+ */
+abstract public class SolrCoreContainerReporter extends SolrMetricReporter {
+
+  protected CoreContainer coreContainer;
+
+  protected SolrCoreContainerReporter(SolrMetricManager metricManager, String registryName) {
+    super(metricManager, registryName);
+  }
+
+  @Override
+  final public void init(PluginInfo pluginInfo) {
+    throw new UnsupportedOperationException(getClass().getCanonicalName()+".init(PluginInfo) is not supported, use init(PluginInfo,CoreContainer) instead.");
+  }
+
+  public void init(PluginInfo pluginInfo, CoreContainer coreContainer) {
+    super.init(pluginInfo);
+    this.coreContainer = coreContainer;
+  }
+
+  public CoreContainer getCoreContainer() {
+    return coreContainer;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a8fbff4d/solr/core/src/java/org/apache/solr/metrics/SolrCoreReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrCoreReporter.java b/solr/core/src/java/org/apache/solr/metrics/SolrCoreReporter.java
new file mode 100644
index 0000000..bd2672c
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrCoreReporter.java
@@ -0,0 +1,47 @@
+/*
+ * 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.metrics;
+
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrCore;
+
+/**
+ * A {@link FilteringSolrMetricReporter} that has access to its {@link SolrCore}.
+ */
+abstract public class SolrCoreReporter extends FilteringSolrMetricReporter {
+
+  protected SolrCore core;
+
+  public SolrCoreReporter(SolrMetricManager metricManager, String registryName) {
+    super(metricManager, registryName);
+  }
+
+  @Override
+  final public void init(PluginInfo pluginInfo) {
+    throw new UnsupportedOperationException(getClass().getCanonicalName()+".init(PluginInfo) is not supported, use init(PluginInfo,SolrCore) instead.");
+  }
+
+  public void init(PluginInfo pluginInfo, SolrCore core) {
+    super.init(pluginInfo);
+    this.core = core;
+  }
+
+  public SolrCore getCore() {
+    return core;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a8fbff4d/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
index adfdda3..4223675 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -54,8 +54,6 @@ import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.metrics.reporters.solr.SolrClusterReporter;
-import org.apache.solr.metrics.reporters.solr.SolrShardReporter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -875,10 +873,10 @@ public class SolrMetricManager {
         new Object[]{this, registry}
     );
     try {
-      if (reporter instanceof SolrShardReporter) {
-        ((SolrShardReporter)reporter).init(pluginInfo, solrCore);
-      } else if (reporter instanceof SolrClusterReporter) {
-        ((SolrClusterReporter)reporter).init(pluginInfo, coreContainer);
+      if (reporter instanceof SolrCoreReporter) {
+        ((SolrCoreReporter)reporter).init(pluginInfo, solrCore);
+      } else if (reporter instanceof SolrCoreContainerReporter) {
+        ((SolrCoreContainerReporter)reporter).init(pluginInfo, coreContainer);
       } else {
         reporter.init(pluginInfo);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a8fbff4d/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java
index 35802dc..081dff0 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java
@@ -36,8 +36,8 @@ import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.handler.admin.MetricsCollectorHandler;
+import org.apache.solr.metrics.SolrCoreContainerReporter;
 import org.apache.solr.metrics.SolrMetricManager;
-import org.apache.solr.metrics.SolrMetricReporter;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -90,7 +90,7 @@ import static org.apache.solr.common.params.CommonParams.ID;
  * </pre>
  *
  */
-public class SolrClusterReporter extends SolrMetricReporter {
+public class SolrClusterReporter extends SolrCoreContainerReporter {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   public static final String CLUSTER_GROUP = SolrMetricManager.overridableRegistryName(SolrInfoBean.Group.cluster.toString());
@@ -170,11 +170,6 @@ public class SolrClusterReporter extends SolrMetricReporter {
   }
 
   @Override
-  public void init(PluginInfo pluginInfo) {
-    throw new UnsupportedOperationException(getClass().getCanonicalName()+".init(PluginInfo) is not supported, use init(PluginInfo,CoreContainer) instead.");
-  }
-
-  @Override
   protected void doInit() {
     if (reports.isEmpty()) { // set defaults
       reports = DEFAULT_REPORTS;
@@ -193,8 +188,9 @@ public class SolrClusterReporter extends SolrMetricReporter {
     }
   }
 
+  @Override
   public void init(PluginInfo pluginInfo, CoreContainer cc) {
-    super.init(pluginInfo);
+    super.init(pluginInfo, cc);
     if (reporter != null) {
       reporter.close();;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a8fbff4d/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
index d07609d..c477e47 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
@@ -31,7 +31,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.admin.MetricsCollectorHandler;
-import org.apache.solr.metrics.FilteringSolrMetricReporter;
+import org.apache.solr.metrics.SolrCoreReporter;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -59,7 +59,7 @@ import com.codahale.metrics.MetricFilter;
  *    &lt;/reporter&gt;
  * </pre>
  */
-public class SolrShardReporter extends FilteringSolrMetricReporter {
+public class SolrShardReporter extends SolrCoreReporter {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   public static final List<String> DEFAULT_FILTERS = new ArrayList(){{
@@ -92,11 +92,6 @@ public class SolrShardReporter extends FilteringSolrMetricReporter {
   }
 
   @Override
-  public void init(PluginInfo pluginInfo) {
-    throw new UnsupportedOperationException(getClass().getCanonicalName()+".init(PluginInfo) is not supported, use init(PluginInfo,SolrCore) instead.");
-  }
-
-  @Override
   protected void doInit() {
     if (filters.isEmpty()) {
       filters = DEFAULT_FILTERS;
@@ -122,8 +117,9 @@ public class SolrShardReporter extends FilteringSolrMetricReporter {
     }
   }
 
+  @Override
   public void init(PluginInfo pluginInfo, SolrCore core) {
-    super.init(pluginInfo);
+    super.init(pluginInfo, core);
     if (reporter != null) {
       reporter.close();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a8fbff4d/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java
index 6b926bf..59952c3 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java
@@ -26,6 +26,8 @@ import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.metrics.AggregateMetric;
+import org.apache.solr.metrics.SolrCoreContainerReporter;
+import org.apache.solr.metrics.SolrCoreReporter;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricReporter;
 import org.apache.solr.metrics.reporters.SolrJmxReporter;
@@ -96,6 +98,9 @@ public class SolrCloudReportersTest extends SolrCloudTestCase {
       assertNotNull(reporter);
       assertTrue(reporter.toString(), reporter instanceof SolrClusterReporter);
       assertEquals(5, reporter.getPeriod());
+      assertTrue(reporter.toString(), reporter instanceof SolrCoreContainerReporter);
+      SolrCoreContainerReporter solrCoreContainerReporter = (SolrCoreContainerReporter)reporter;
+      assertNotNull(solrCoreContainerReporter.getCoreContainer());
       for (String registryName : metricManager.registryNames(".*\\.shard[0-9]\\.replica.*")) {
         reporters = metricManager.getReporters(registryName);
         jmxReporter = 0;
@@ -114,6 +119,9 @@ public class SolrCloudReportersTest extends SolrCloudTestCase {
         assertNotNull(reporter);
         assertTrue(reporter.toString(), reporter instanceof SolrShardReporter);
         assertEquals(5, reporter.getPeriod());
+        assertTrue(reporter.toString(), reporter instanceof SolrCoreReporter);
+        SolrCoreReporter solrCoreReporter = (SolrCoreReporter)reporter;
+        assertNotNull(solrCoreReporter.getCore());
       }
       for (String registryName : metricManager.registryNames(".*\\.leader")) {
         leaderRegistries++;