You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2020/09/04 08:22:24 UTC

[lucene-solr] 01/02: preload conf

This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch jira/solr14827
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 46e61913b2613e5834ff7c82cad6ebc3f39b3f08
Author: noblepaul <no...@gmail.com>
AuthorDate: Fri Sep 4 16:47:00 2020 +1000

    preload conf
---
 .../java/org/apache/solr/core/XmlConfigFile.java   |   8 --
 .../apache/solr/schema/FieldTypePluginLoader.java  |  10 +-
 .../java/org/apache/solr/schema/IndexSchema.java   |  14 ++-
 .../java/org/apache/solr/util/DOMConfigNode.java   |   2 +
 .../java/org/apache/solr/util/DataConfigNode.java  | 106 +++++++++++++++++++++
 .../solr/util/plugin/AbstractPluginLoader.java     |   2 -
 .../java/org/apache/solr/common/ConfigNode.java    |  36 +++++--
 7 files changed, 148 insertions(+), 30 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java b/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
index 68121b9..08fe569 100644
--- a/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
+++ b/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
@@ -44,10 +44,8 @@ import java.util.TreeSet;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
-import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.XMLErrorLogger;
-import org.apache.solr.util.DOMConfigNode;
 import org.apache.solr.util.DOMUtil;
 import org.apache.solr.util.SystemIdResolver;
 import org.slf4j.Logger;
@@ -74,8 +72,6 @@ public class XmlConfigFile { // formerly simply "Config"
   private final String prefix;
   private final String name;
   private final SolrResourceLoader loader;
-
-  private final ConfigNode rootNode;
   private final Properties substituteProperties;
   private int zkVersion = -1;
 
@@ -157,15 +153,11 @@ public class XmlConfigFile { // formerly simply "Config"
       if (substituteProps != null) {
         DOMUtil.substituteProperties(doc, getSubstituteProperties());
       }
-      rootNode = new DOMConfigNode(doc.getDocumentElement());
     } catch (ParserConfigurationException | SAXException | TransformerException e)  {
       SolrException.log(log, "Exception during parsing file: " + name, e);
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
     }
   }
-  public ConfigNode rootNode() {
-    return rootNode;
-  }
 
   /*
      * Assert that assertCondition is true.
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java b/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
index 7a4e00c..b64beb5 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
@@ -80,23 +80,23 @@ public final class FieldTypePluginLoader
     ft.setTypeName(name);
     
     String expression = "./analyzer[@type='query']";
-    ConfigNode anode = node.child(it -> "analyzer".equals(it.name()) && "query".equals(it.attributes().get("type")));
+    ConfigNode anode = node.child(it -> "query".equals(it.attributes().get("type")) , "analyzer");
     Analyzer queryAnalyzer = readAnalyzer(anode);
 
 //    expression = "./analyzer[@type='multiterm']";
-    anode = node.child(it -> "analyzer".equals(it.name()) && "multiterm".equals(it.attributes().get("type"))) ;
+    anode = node.child(it -> "multiterm".equals(it.attributes().get("type") ), "analyzer");
     Analyzer multiAnalyzer = readAnalyzer(anode);
 
     // An analyzer without a type specified, or with type="index"
     expression = "./analyzer[not(@type)] | ./analyzer[@type='index']";
-    anode = node.child(it -> "analyzer".equals(it.name()) &&
-        (it.attributes().get("type") == null || "index".equals(it.attributes().get("type"))));
+    anode = node.child(it ->
+        (it.attributes().get("type") == null || "index".equals(it.attributes().get("type"))), "analyzer");
 //    anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
     Analyzer analyzer = readAnalyzer(anode);
 
     // a custom similarity[Factory]
     expression = "./similarity";
-    anode = node.child(it -> "similarity".equals(it.name()) && "multiterm".equals(it.attributes().get("type"))) ;
+    anode = node.child(it -> "multiterm".equals(it.attributes().get("type")), "similarity") ;
 
     SimilarityFactory simFactory = IndexSchema.readSimilarity(loader, anode);
     if (null != simFactory) {
diff --git a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
index 74abfcd..59d910a 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -73,7 +73,9 @@ import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.similarities.SchemaSimilarityFactory;
 import org.apache.solr.uninverting.UninvertingReader;
 import org.apache.solr.util.ConcurrentLRUCache;
+import org.apache.solr.util.DOMConfigNode;
 import org.apache.solr.util.DOMUtil;
+import org.apache.solr.util.DataConfigNode;
 import org.apache.solr.util.PayloadUtils;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
@@ -144,7 +146,8 @@ public class IndexSchema {
 
   public DynamicField[] getDynamicFields() { return dynamicFields; }
 
-  private final Set<String> FIELDTYPE_KEYS = ImmutableSet.of("fieldtype", "fieldType");
+  private static final Set<String> FIELDTYPE_KEYS = ImmutableSet.of("fieldtype", "fieldType");
+  private static final Set<String> FIELD_KEYS = ImmutableSet.of("dynamicField", "field");
 
   @SuppressWarnings({"unchecked", "rawtypes"})
   protected Cache<String, SchemaField> dynamicFieldCache = new ConcurrentLRUCache(10000, 8000, 9000,100, false,false, null);
@@ -166,6 +169,8 @@ public class IndexSchema {
    * directives that target them.
    */
   protected Map<SchemaField, Integer> copyFieldTargetCounts = new HashMap<>();
+//  static AtomicLong totalSchemaLoadTime = new AtomicLong();
+
 
   /**
    * Constructs a schema using the specified resource name and stream.
@@ -483,7 +488,7 @@ public class IndexSchema {
 //      final XPath xpath = schemaConf.getXPath();
 //      String expression = stepsToPath(SCHEMA, AT + NAME);
 //      Node nd = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
-      ConfigNode rootNode = schemaConf.rootNode();
+      ConfigNode rootNode = new DataConfigNode(new DOMConfigNode(schemaConf.getDocument().getDocumentElement()) , Collections.singleton("similarity")) ;
       name = rootNode.attributes().get("name");
       StringBuilder sb = new StringBuilder();
       // Another case where the initialization from the test harness is different than the "real world"
@@ -508,7 +513,7 @@ public class IndexSchema {
 //      NodeList nodes = (NodeList) xpath.evaluate(expression, document, XPathConstants.NODESET);
 
       typeLoader.load(solrClassLoader,
-          rootNode.children(it -> FIELDTYPE_KEYS.contains( it.name())));
+          rootNode.children(null, FIELDTYPE_KEYS));
 
       // load the fields
       Map<String,Boolean> explicitRequiredProp = loadFields(rootNode);
@@ -657,8 +662,7 @@ public class IndexSchema {
         + XPATH_OR + stepsToPath(SCHEMA, DYNAMIC_FIELD)
         + XPATH_OR + stepsToPath(SCHEMA, FIELDS, FIELD)
         + XPATH_OR + stepsToPath(SCHEMA, FIELDS, DYNAMIC_FIELD);*/
-    List<ConfigNode> nodes = n.children(it -> "field".equals(it.name()) ||
-        "dynamicField".equals(it.name()));
+    List<ConfigNode> nodes = n.children(null,  FIELD_KEYS);
 
 //    NodeList nodes = (NodeList)xpath.evaluate(expression, document, XPathConstants.NODESET);
 
diff --git a/solr/core/src/java/org/apache/solr/util/DOMConfigNode.java b/solr/core/src/java/org/apache/solr/util/DOMConfigNode.java
index 1cccbce..85ca9f7 100644
--- a/solr/core/src/java/org/apache/solr/util/DOMConfigNode.java
+++ b/solr/core/src/java/org/apache/solr/util/DOMConfigNode.java
@@ -35,6 +35,7 @@ public class DOMConfigNode implements ConfigNode {
   private final Node node;
   SimpleMap<String> attrs;
 
+
   @Override
   public String name() {
     return node.getNodeName();
@@ -83,4 +84,5 @@ public class DOMConfigNode implements ConfigNode {
       if (Boolean.FALSE == toContinue) break;
     }
   }
+
 }
diff --git a/solr/core/src/java/org/apache/solr/util/DataConfigNode.java b/solr/core/src/java/org/apache/solr/util/DataConfigNode.java
new file mode 100644
index 0000000..2567667
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/DataConfigNode.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import org.apache.solr.cluster.api.SimpleMap;
+import org.apache.solr.common.ConfigNode;
+
+public class DataConfigNode implements ConfigNode {
+  final String name;
+  final SimpleMap<String> attributes;
+  private final Map<String, List<ConfigNode>> kids = new HashMap<>();
+  private String textData;
+
+  public DataConfigNode(ConfigNode root, Set<String> ignore) {
+    name = root.name();
+    attributes = root.attributes();
+    textData = root.textValue();
+    if (textData != null) textData = textData.trim();
+    root.forEachChild(it -> {
+      List<ConfigNode> nodes = kids.computeIfAbsent(it.name(),
+          k -> new ArrayList<>());
+      if (ignore != null && ignore.contains(it.name())) {
+        nodes.add(it);
+      } else {
+        nodes.add(new DataConfigNode(it, ignore));
+      }
+      return Boolean.TRUE;
+    });
+
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  public String textValue() {
+    return textData;
+  }
+
+  @Override
+  public SimpleMap<String> attributes() {
+    return attributes;
+  }
+
+  @Override
+  public ConfigNode child(String name) {
+    List<ConfigNode> val = kids.get(name);
+    return val == null || val.isEmpty() ? null : val.get(0);
+  }
+
+  @Override
+  public List<ConfigNode> children(String name) {
+    return kids.getOrDefault(name, Collections.emptyList());
+  }
+
+  @Override
+  public List<ConfigNode> children(Predicate<ConfigNode> test, Set<String> set) {
+    List<ConfigNode> result = new ArrayList<>();
+    for (String s : set) {
+      List<ConfigNode> vals = kids.get(s);
+      if (vals != null) {
+        vals.forEach(it -> {
+          if (test == null || test.test(it)) {
+            result.add(it);
+          }
+        });
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public void forEachChild(Function<ConfigNode, Boolean> fun) {
+    kids.forEach((s, configNodes) -> {
+      if (configNodes != null) {
+        configNodes.forEach(fun::apply);
+      }
+    });
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java b/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
index 38a8af0..0eb0fd0 100644
--- a/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
@@ -29,8 +29,6 @@ import org.apache.solr.common.cloud.SolrClassLoader;
 import org.apache.solr.util.DOMUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
 
 import static org.apache.solr.common.params.CommonParams.NAME;
 
diff --git a/solr/solrj/src/java/org/apache/solr/common/ConfigNode.java b/solr/solrj/src/java/org/apache/solr/common/ConfigNode.java
index 90b0b81..f046440 100644
--- a/solr/solrj/src/java/org/apache/solr/common/ConfigNode.java
+++ b/solr/solrj/src/java/org/apache/solr/common/ConfigNode.java
@@ -17,7 +17,9 @@
 package org.apache.solr.common;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
+import java.util.Set;
 import java.util.function.Function;
 import java.util.function.Predicate;
 
@@ -28,26 +30,33 @@ import org.apache.solr.cluster.api.SimpleMap;
  */
 public interface ConfigNode {
 
-  /**Name of the tag
+  /**
+   * Name of the tag
    */
   String name();
 
-  /**Text value of the node
+  /**
+   * Text value of the node
    */
   String textValue();
 
-  /**Attributes
+  /**
+   * Attributes
    */
   SimpleMap<String> attributes();
 
-  /** Child by name
+  /**
+   * Child by name
    */
-  ConfigNode child(String name);
+  default ConfigNode child(String name) {
+    return child(null, name);
+  }
 
-  default ConfigNode child(Predicate<ConfigNode> test) {
+  default ConfigNode child(Predicate<ConfigNode> test, String name) {
     ConfigNode[] result = new ConfigNode[1];
     forEachChild(it -> {
-      if (test.test(it)) {
+      if (name!=null && !name.equals(it.name())) return Boolean.TRUE;
+      if (test == null || test.test(it)) {
         result[0] = it;
         return Boolean.FALSE;
       }
@@ -56,16 +65,23 @@ public interface ConfigNode {
     return result[0];
   }
 
-  default List<ConfigNode> children(Predicate<ConfigNode> test) {
+  default List<ConfigNode> children(Predicate<ConfigNode> test, String... nodeNames) {
+    return children(test, nodeNames == null ? Collections.emptySet() : Set.of(nodeNames));
+  }
+
+  default List<ConfigNode> children(Predicate<ConfigNode> test, Set<String> set) {
     List<ConfigNode> result = new ArrayList<>();
     forEachChild(it -> {
-      if (test.test(it)) result.add(it);
+      if (set != null && !set.isEmpty() && !set.contains(it.name())) return Boolean.TRUE;
+      if (test == null || test.test(it)) result.add(it);
       return Boolean.TRUE;
     });
     return result;
   }
 
-  List<ConfigNode> children(String name);
+  default List<ConfigNode> children(String name) {
+    return children(null, Collections.singleton(name));
+  }
 
   void forEachChild(Function<ConfigNode, Boolean> fun);