You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by is...@apache.org on 2020/12/05 04:37:28 UTC

[lucene-solr] 01/18: SOLR-14827: Refactor schema loading to not use XPath

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

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

commit 74c98f04e6dfbe848d2462e51927d4ddda975ae0
Author: noblepaul <no...@gmail.com>
AuthorDate: Fri Sep 4 01:04:49 2020 +1000

    SOLR-14827: Refactor schema loading to not use XPath
---
 .../java/org/apache/solr/core/XmlConfigFile.java   |   9 ++
 .../apache/solr/schema/FieldTypePluginLoader.java  |  85 +++++++------
 .../java/org/apache/solr/schema/IndexSchema.java   | 134 +++++++++++----------
 .../org/apache/solr/schema/ManagedIndexSchema.java |   4 +-
 .../java/org/apache/solr/util/DOMConfigNode.java   |  86 +++++++++++++
 .../solr/util/plugin/AbstractPluginLoader.java     |  18 +--
 .../apache/solr/util/plugin/MapPluginLoader.java   |   6 +-
 .../solr/util/plugin/NamedListPluginLoader.java    |   4 +-
 .../org/apache/solr/cluster/api/SimpleMap.java     |   5 +
 .../java/org/apache/solr/common/ConfigNode.java    |  73 +++++++++++
 .../java/org/apache/solr/common/util/DOMUtil.java  | 108 +++++++++++++----
 11 files changed, 388 insertions(+), 144 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 6d8cb92..ed28075 100644
--- a/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
+++ b/solr/core/src/java/org/apache/solr/core/XmlConfigFile.java
@@ -44,9 +44,12 @@ 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.DOMUtil;
 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;
 import org.slf4j.LoggerFactory;
@@ -72,6 +75,8 @@ 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;
 
@@ -152,11 +157,15 @@ 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 b4f51ec..2d86113 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
@@ -16,13 +16,10 @@
  */
 package org.apache.solr.schema;
 
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
-import javax.xml.xpath.XPathFactory;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
@@ -33,6 +30,7 @@ import org.apache.lucene.analysis.TokenizerFactory;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.util.Version;
 import org.apache.solr.analysis.TokenizerChain;
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.SolrClassLoader;
 import org.apache.solr.common.util.DOMUtil;
@@ -40,9 +38,6 @@ import org.apache.solr.core.SolrConfig;
 import org.apache.solr.util.plugin.AbstractPluginLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.w3c.dom.NamedNodeMap;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
 
 import static org.apache.solr.common.params.CommonParams.NAME;
 
@@ -54,8 +49,6 @@ public final class FieldTypePluginLoader
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final XPath xpath = XPathFactory.newInstance().newXPath();
-
   /**
    * @param schema The schema that will be used to initialize the FieldTypes
    * @param fieldTypes All FieldTypes that are instantiated by 
@@ -81,27 +74,30 @@ public final class FieldTypePluginLoader
   protected FieldType create( SolrClassLoader loader,
                               String name, 
                               String className,
-                              Node node ) throws Exception {
+                              ConfigNode node ) throws Exception {
 
     FieldType ft = loader.newInstance(className, FieldType.class);
     ft.setTypeName(name);
     
     String expression = "./analyzer[@type='query']";
-    Node anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
+    ConfigNode anode = node.child(it -> "analyzer".equals(it.name()) && "query".equals(it.attributes().get("type")));
     Analyzer queryAnalyzer = readAnalyzer(anode);
 
-    expression = "./analyzer[@type='multiterm']";
-    anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
+//    expression = "./analyzer[@type='multiterm']";
+    anode = node.child(it -> "analyzer".equals(it.name()) && "multiterm".equals(it.attributes().get("type"))) ;
     Analyzer multiAnalyzer = readAnalyzer(anode);
 
     // An analyzer without a type specified, or with type="index"
     expression = "./analyzer[not(@type)] | ./analyzer[@type='index']";
-    anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
+    anode = node.child(it -> "analyzer".equals(it.name()) &&
+        (it.attributes().get("type") == null || "index".equals(it.attributes().get("type"))));
+//    anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
     Analyzer analyzer = readAnalyzer(anode);
 
     // a custom similarity[Factory]
     expression = "./similarity";
-    anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
+    anode = node.child(it -> "similarity".equals(it.name()) && "multiterm".equals(it.attributes().get("type"))) ;
+
     SimilarityFactory simFactory = IndexSchema.readSimilarity(loader, anode);
     if (null != simFactory) {
       ft.setSimilarity(simFactory);
@@ -153,9 +149,9 @@ public final class FieldTypePluginLoader
   }
   
   @Override
-  protected void init(FieldType plugin, Node node) throws Exception {
+  protected void init(FieldType plugin, ConfigNode node) throws Exception {
 
-    Map<String, String> params = DOMUtil.toMapExcept(node.getAttributes(), NAME);
+    Map<String, String> params = DOMUtil.toMapExcept(node, NAME);
     plugin.setArgs(schema, params);
   }
 
@@ -187,7 +183,7 @@ public final class FieldTypePluginLoader
   // <analyzer><tokenizer class="...."/><tokenizer class="...." arg="....">
   //
   //
-  private Analyzer readAnalyzer(Node node) throws XPathExpressionException {
+  private Analyzer readAnalyzer(ConfigNode node)  {
                                 
     final SolrClassLoader loader = schema.getSolrClassLoader();
 
@@ -196,26 +192,29 @@ public final class FieldTypePluginLoader
     // Node node = DOMUtil.getChild(fieldtype,"analyzer");
     
     if (node == null) return null;
-    NamedNodeMap attrs = node.getAttributes();
-    String analyzerName = DOMUtil.getAttr(attrs,"class");
+//    NamedNodeMap attrs = node.getAttributes();
+    String analyzerName = DOMUtil.getAttr(node,"class", null);
 
     // check for all of these up front, so we can error if used in 
     // conjunction with an explicit analyzer class.
-    NodeList charFilterNodes = (NodeList)xpath.evaluate
-      ("./charFilter",  node, XPathConstants.NODESET);
-    NodeList tokenizerNodes = (NodeList)xpath.evaluate
-      ("./tokenizer", node, XPathConstants.NODESET);
-    NodeList tokenFilterNodes = (NodeList)xpath.evaluate
-      ("./filter", node, XPathConstants.NODESET);
-      
+//    NodeList charFilterNodes = (NodeList)xpath.evaluate
+//      ("./charFilter",  node, XPathConstants.NODESET);
+//    NodeList tokenizerNodes = (NodeList)xpath.evaluate
+//      ("./tokenizer", node, XPathConstants.NODESET);
+//    NodeList tokenFilterNodes = (NodeList)xpath.evaluate
+//      ("./filter", node, XPathConstants.NODESET);
+    List<ConfigNode> charFilterNodes = node.children("charFilter");
+    List<ConfigNode> tokenizerNodes = node.children("tokenizer");
+    List<ConfigNode> tokenFilterNodes = node.children("filter");
+
     if (analyzerName != null) {
 
       // explicitly check for child analysis factories instead of
       // just any child nodes, because the user might have their
       // own custom nodes (ie: <description> or something like that)
-      if (0 != charFilterNodes.getLength() ||
-          0 != tokenizerNodes.getLength() ||
-          0 != tokenFilterNodes.getLength()) {
+      if (0 != charFilterNodes.size() ||
+          0 != tokenizerNodes.size() ||
+          0 != tokenFilterNodes.size()) {
         throw new SolrException
         ( SolrException.ErrorCode.SERVER_ERROR,
           "Configuration Error: Analyzer class='" + analyzerName +
@@ -227,7 +226,7 @@ public final class FieldTypePluginLoader
         final Class<? extends Analyzer> clazz = loader.findClass(analyzerName, Analyzer.class);
         Analyzer analyzer = clazz.getConstructor().newInstance();
 
-        final String matchVersionStr = DOMUtil.getAttr(attrs, LUCENE_MATCH_VERSION_PARAM);
+        final String matchVersionStr = DOMUtil.getAttr(node, LUCENE_MATCH_VERSION_PARAM,null);
         final Version luceneMatchVersion = (matchVersionStr == null) ?
           schema.getDefaultLuceneMatchVersion() :
           SolrConfig.parseLuceneVersionString(matchVersionStr);
@@ -254,9 +253,9 @@ public final class FieldTypePluginLoader
       ("[schema.xml] analyzer/charFilter", CharFilterFactory.class, false, false) {
 
       @Override
-      @SuppressWarnings({"rawtypes"})
-      protected CharFilterFactory create(SolrClassLoader loader, String name, String className, Node node) throws Exception {
-        final Map<String,String> params = DOMUtil.toMap(node.getAttributes());
+      @SuppressWarnings("rawtypes")
+      protected CharFilterFactory create(SolrClassLoader loader, String name, String className, ConfigNode node) throws Exception {
+        final Map<String,String> params = DOMUtil.toMapExcept(node);
         String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
         params.put(LUCENE_MATCH_VERSION_PARAM, parseConfiguredVersion(configuredVersion, CharFilterFactory.class.getSimpleName()).toString());
         CharFilterFactory factory;
@@ -279,7 +278,7 @@ public final class FieldTypePluginLoader
       }
 
       @Override
-      protected void init(CharFilterFactory plugin, Node node) throws Exception {
+      protected void init(CharFilterFactory plugin, ConfigNode node) throws Exception {
         if( plugin != null ) {
           charFilters.add( plugin );
         }
@@ -305,9 +304,9 @@ public final class FieldTypePluginLoader
       ("[schema.xml] analyzer/tokenizer", TokenizerFactory.class, false, false) {
       
       @Override
-      @SuppressWarnings({"rawtypes"})
-      protected TokenizerFactory create(SolrClassLoader loader, String name, String className, Node node) throws Exception {
-        final Map<String,String> params = DOMUtil.toMap(node.getAttributes());
+      @SuppressWarnings("rawtypes")
+      protected TokenizerFactory create(SolrClassLoader loader, String name, String className, ConfigNode node) throws Exception {
+        final Map<String,String> params = DOMUtil.toMap(node);
         String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
         params.put(LUCENE_MATCH_VERSION_PARAM, parseConfiguredVersion(configuredVersion, TokenizerFactory.class.getSimpleName()).toString());
         TokenizerFactory factory;
@@ -330,7 +329,7 @@ public final class FieldTypePluginLoader
       }
       
       @Override
-      protected void init(TokenizerFactory plugin, Node node) throws Exception {
+      protected void init(TokenizerFactory plugin, ConfigNode node) throws Exception {
         if( !tokenizers.isEmpty() ) {
           throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
               "The schema defines multiple tokenizers for: "+node );
@@ -360,9 +359,9 @@ public final class FieldTypePluginLoader
       new AbstractPluginLoader<TokenFilterFactory>("[schema.xml] analyzer/filter", TokenFilterFactory.class, false, false)
     {
       @Override
-      @SuppressWarnings({"rawtypes"})
-      protected TokenFilterFactory create(SolrClassLoader loader, String name, String className, Node node) throws Exception {
-        final Map<String,String> params = DOMUtil.toMap(node.getAttributes());
+      @SuppressWarnings("rawtypes")
+      protected TokenFilterFactory create(SolrClassLoader loader, String name, String className, ConfigNode node) throws Exception {
+        final Map<String,String> params = DOMUtil.toMap(node);
         String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
         params.put(LUCENE_MATCH_VERSION_PARAM, parseConfiguredVersion(configuredVersion, TokenFilterFactory.class.getSimpleName()).toString());
         TokenFilterFactory factory;
@@ -385,7 +384,7 @@ public final class FieldTypePluginLoader
       }
       
       @Override
-      protected void init(TokenFilterFactory plugin, Node node) throws Exception {
+      protected void init(TokenFilterFactory plugin, ConfigNode node) throws Exception {
         if( plugin != null ) {
           filters.add( plugin );
         }
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 305cd50..c451176 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -16,8 +16,6 @@
  */
 package org.apache.solr.schema;
 
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathConstants;
 import javax.xml.xpath.XPathExpressionException;
 import java.io.IOException;
 import java.io.Writer;
@@ -39,17 +37,20 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Function;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import com.google.common.collect.ImmutableSet;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.DelegatingAnalyzerWrapper;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.queries.payloads.PayloadDecoder;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.Version;
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.MapSerializable;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrException;
@@ -77,11 +78,6 @@ import org.apache.solr.util.PayloadUtils;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.NamedNodeMap;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import static java.util.Arrays.asList;
@@ -148,6 +144,8 @@ public class IndexSchema {
 
   public DynamicField[] getDynamicFields() { return dynamicFields; }
 
+  private final Set<String> FIELDTYPE_KEYS = ImmutableSet.of("fieldtype", "fieldType");
+
   @SuppressWarnings({"unchecked", "rawtypes"})
   protected Cache<String, SchemaField> dynamicFieldCache = new ConcurrentLRUCache(10000, 8000, 9000,100, false,false, null);
 
@@ -163,7 +161,6 @@ public class IndexSchema {
   public DynamicCopy[] getDynamicCopyFields() { return dynamicCopyFields; }
 
   private Map<FieldType, PayloadDecoder> decoders = new HashMap<>();  // cache to avoid scanning token filters repeatedly, unnecessarily
-
   /**
    * keys are all fields copied to, count is num of copyField
    * directives that target them.
@@ -180,7 +177,9 @@ public class IndexSchema {
 
     this.resourceName = Objects.requireNonNull(name);
     try {
+//      long start = System.currentTimeMillis();
       readSchema(is);
+//      System.out.println("schema-load-time : "+ totalSchemaLoadTime.addAndGet (System.currentTimeMillis() - start));
       loader.inform(loader);
     } catch (IOException e) {
       throw new RuntimeException(e);
@@ -480,17 +479,18 @@ public class IndexSchema {
       // pass the config resource loader to avoid building an empty one for no reason:
       // in the current case though, the stream is valid so we wont load the resource by name
       XmlConfigFile schemaConf = new XmlConfigFile(loader, SCHEMA, is, SLASH+SCHEMA+SLASH, substitutableProperties);
-      Document document = schemaConf.getDocument();
-      final XPath xpath = schemaConf.getXPath();
-      String expression = stepsToPath(SCHEMA, AT + NAME);
-      Node nd = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
+//      Document document = schemaConf.getDocument();
+//      final XPath xpath = schemaConf.getXPath();
+//      String expression = stepsToPath(SCHEMA, AT + NAME);
+//      Node nd = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
+      ConfigNode rootNode = schemaConf.rootNode();
+      name = rootNode.attributes().get("name");
       StringBuilder sb = new StringBuilder();
       // Another case where the initialization from the test harness is different than the "real world"
-      if (nd==null) {
+      if (name==null) {
         sb.append("schema has no name!");
         log.warn("{}", sb);
       } else {
-        name = nd.getNodeValue();
         sb.append("Schema ");
         sb.append(NAME);
         sb.append("=");
@@ -499,21 +499,24 @@ public class IndexSchema {
       }
 
       //                      /schema/@version
-      expression = stepsToPath(SCHEMA, AT + VERSION);
-      version = schemaConf.getFloat(expression, 1.0f);
+//      expression = stepsToPath(SCHEMA, AT + VERSION);
+      version =  Float.parseFloat(rootNode.attributes().get("version","1.0f"));
 
       // load the Field Types
       final FieldTypePluginLoader typeLoader = new FieldTypePluginLoader(this, fieldTypes, schemaAware);
-      expression = getFieldTypeXPathExpressions();
-      NodeList nodes = (NodeList) xpath.evaluate(expression, document, XPathConstants.NODESET);
-      typeLoader.load(solrClassLoader, nodes);
+//      expression = getFieldTypeXPathExpressions();
+//      NodeList nodes = (NodeList) xpath.evaluate(expression, document, XPathConstants.NODESET);
+
+      typeLoader.load(solrClassLoader,
+          rootNode.children(it -> FIELDTYPE_KEYS.contains( it.name())));
 
       // load the fields
-      Map<String,Boolean> explicitRequiredProp = loadFields(document, xpath);
+      Map<String,Boolean> explicitRequiredProp = loadFields(rootNode);
+
+//      expression = stepsToPath(SCHEMA, SIMILARITY); //   /schema/similarity
+//      Node node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
 
-      expression = stepsToPath(SCHEMA, SIMILARITY); //   /schema/similarity
-      Node node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
-      similarityFactory = readSimilarity(solrClassLoader, node);
+      similarityFactory = readSimilarity(solrClassLoader, rootNode.child(SIMILARITY));
       if (similarityFactory == null) {
         final Class<?> simClass = SchemaSimilarityFactory.class;
         // use the loader to ensure proper SolrCoreAware handling
@@ -537,26 +540,30 @@ public class IndexSchema {
       }
 
       //                      /schema/defaultSearchField/text()
-      expression = stepsToPath(SCHEMA, "defaultSearchField", TEXT_FUNCTION);
-      node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
+//      expression = stepsToPath(SCHEMA, "defaultSearchField", TEXT_FUNCTION);
+//      node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
+      ConfigNode node = rootNode.child("defaultSearchField");
       if (node != null) {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Setting defaultSearchField in schema not supported since Solr 7");
       }
+      node = rootNode.child("solrQueryParser");
 
       //                      /schema/solrQueryParser/@defaultOperator
-      expression = stepsToPath(SCHEMA, "solrQueryParser", AT + "defaultOperator");
-      node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
+//      expression = stepsToPath(SCHEMA, "solrQueryParser", AT + "defaultOperator");
+//      node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
       if (node != null) {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Setting default operator in schema (solrQueryParser/@defaultOperator) not supported");
       }
 
       //                      /schema/uniqueKey/text()
-      expression = stepsToPath(SCHEMA, UNIQUE_KEY, TEXT_FUNCTION);
-      node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
+//      expression = stepsToPath(SCHEMA, UNIQUE_KEY, TEXT_FUNCTION);
+//      node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
+      node = rootNode.child(UNIQUE_KEY);
+
       if (node==null) {
         log.warn("no {} specified in schema.", UNIQUE_KEY);
       } else {
-        uniqueKeyField=getIndexedField(node.getNodeValue().trim());
+        uniqueKeyField=getIndexedField(node.textValue().trim());
         uniqueKeyFieldName=uniqueKeyField.getName();
         uniqueKeyFieldType=uniqueKeyField.getType();
         
@@ -608,7 +615,7 @@ public class IndexSchema {
       // expression = "/schema/copyField";
 
       dynamicCopyFields = new DynamicCopy[] {};
-      loadCopyFields(document, xpath);
+      loadCopyFields(rootNode);
 
       postReadInform();
 
@@ -639,61 +646,61 @@ public class IndexSchema {
    * 
    * @return a map from field name to explicit required value  
    */ 
-  protected synchronized Map<String,Boolean> loadFields(Document document, XPath xpath) throws XPathExpressionException {
+  protected synchronized Map<String,Boolean> loadFields(ConfigNode n) {
     // Hang on to the fields that say if they are required -- this lets us set a reasonable default for the unique key
     Map<String,Boolean> explicitRequiredProp = new HashMap<>();
     
     ArrayList<DynamicField> dFields = new ArrayList<>();
 
     //                  /schema/field | /schema/dynamicField | /schema/fields/field | /schema/fields/dynamicField
-    String expression = stepsToPath(SCHEMA, FIELD)
+ /*   String expression = stepsToPath(SCHEMA, FIELD)
         + XPATH_OR + stepsToPath(SCHEMA, DYNAMIC_FIELD)
         + XPATH_OR + stepsToPath(SCHEMA, FIELDS, FIELD)
-        + XPATH_OR + stepsToPath(SCHEMA, FIELDS, DYNAMIC_FIELD);
-
-    NodeList nodes = (NodeList)xpath.evaluate(expression, document, XPathConstants.NODESET);
+        + XPATH_OR + stepsToPath(SCHEMA, FIELDS, DYNAMIC_FIELD);*/
+    List<ConfigNode> nodes = n.children(it -> "field".equals(it.name()) ||
+        "dynamicField".equals(it.name()));
 
-    for (int i=0; i<nodes.getLength(); i++) {
-      Node node = nodes.item(i);
+//    NodeList nodes = (NodeList)xpath.evaluate(expression, document, XPathConstants.NODESET);
 
-      NamedNodeMap attrs = node.getAttributes();
+    for (ConfigNode node : nodes) {
+      //      SimpleMap<String> attrs = node.attributes();
 
-      String name = DOMUtil.getAttr(attrs, NAME, "field definition");
+      String name = DOMUtil.getAttr(node, NAME, "field definition");
       log.trace("reading field def {}", name);
-      String type = DOMUtil.getAttr(attrs, TYPE, "field " + name);
+      String type = DOMUtil.getAttr(node, TYPE, "field " + name);
 
       FieldType ft = fieldTypes.get(type);
-      if (ft==null) {
+      if (ft == null) {
         throw new SolrException
             (ErrorCode.BAD_REQUEST, "Unknown " + FIELD_TYPE + " '" + type + "' specified on field " + name);
       }
 
-      Map<String,String> args = DOMUtil.toMapExcept(attrs, NAME, TYPE);
+      Map<String, String> args = DOMUtil.toMapExcept(node, NAME, TYPE);
       if (null != args.get(REQUIRED)) {
         explicitRequiredProp.put(name, Boolean.valueOf(args.get(REQUIRED)));
       }
 
-      SchemaField f = SchemaField.create(name,ft,args);
+      SchemaField f = SchemaField.create(name, ft, args);
 
-      if (node.getNodeName().equals(FIELD)) {
-        SchemaField old = fields.put(f.getName(),f);
-        if( old != null ) {
+      if (node.name().equals(FIELD)) {
+        SchemaField old = fields.put(f.getName(), f);
+        if (old != null) {
           String msg = "[schema.xml] Duplicate field definition for '"
-            + f.getName() + "' [[["+old.toString()+"]]] and [[["+f.toString()+"]]]";
-          throw new SolrException(ErrorCode.SERVER_ERROR, msg );
+              + f.getName() + "' [[[" + old.toString() + "]]] and [[[" + f.toString() + "]]]";
+          throw new SolrException(ErrorCode.SERVER_ERROR, msg);
         }
         log.debug("field defined: {}", f);
-        if( f.getDefaultValue() != null ) {
+        if (f.getDefaultValue() != null) {
           if (log.isDebugEnabled()) {
             log.debug("{} contains default value {}", name, f.getDefaultValue());
           }
-          fieldsWithDefaultValue.add( f );
+          fieldsWithDefaultValue.add(f);
         }
         if (f.isRequired()) {
           log.debug("{} is required in this schema", name);
           requiredFields.add(f);
         }
-      } else if (node.getNodeName().equals(DYNAMIC_FIELD)) {
+      } else if (node.name().equals(DYNAMIC_FIELD)) {
         if (isValidDynamicField(dFields, f)) {
           addDynamicFieldNoDupCheck(dFields, f);
         }
@@ -733,17 +740,18 @@ public class IndexSchema {
   /**
    * Loads the copy fields
    */
-  protected synchronized void loadCopyFields(Document document, XPath xpath) throws XPathExpressionException {
-    String expression = "//" + COPY_FIELD;
-    NodeList nodes = (NodeList)xpath.evaluate(expression, document, XPathConstants.NODESET);
+  protected synchronized void loadCopyFields(ConfigNode n) throws XPathExpressionException {
+//    String expression = "//" + COPY_FIELD;
+//    NodeList nodes = (NodeList)xpath.evaluate(expression, document, XPathConstants.NODESET);
 
-    for (int i=0; i<nodes.getLength(); i++) {
-      Node node = nodes.item(i);
-      NamedNodeMap attrs = node.getAttributes();
+    List<ConfigNode> nodes = n.children(COPY_FIELD);
+    for (int i=0; i<nodes.size(); i++) {
+      ConfigNode node = nodes.get(i);
+//      NamedNodeMap attrs = node.getAttributes();
 
-      String source = DOMUtil.getAttr(attrs, SOURCE, COPY_FIELD + " definition");
-      String dest   = DOMUtil.getAttr(attrs, DESTINATION,  COPY_FIELD + " definition");
-      String maxChars = DOMUtil.getAttr(attrs, MAX_CHARS);
+      String source = DOMUtil.getAttr(node, SOURCE, COPY_FIELD + " definition");
+      String dest   = DOMUtil.getAttr(node, DESTINATION,  COPY_FIELD + " definition");
+      String maxChars = DOMUtil.getAttr(node, MAX_CHARS, null);
 
       int maxCharsInt = CopyField.UNLIMITED;
       if (maxChars != null) {
@@ -990,12 +998,12 @@ public class IndexSchema {
     dynamicCopyFields = temp;
   }
 
-  static SimilarityFactory readSimilarity(SolrClassLoader loader, Node node) {
+  static SimilarityFactory readSimilarity(SolrClassLoader loader, ConfigNode node) {
     if (node==null) {
       return null;
     } else {
       SimilarityFactory similarityFactory;
-      final String classArg = ((Element) node).getAttribute(SimilarityFactory.CLASS_NAME);
+      final String classArg = node.attributes().get(SimilarityFactory.CLASS_NAME);
       final Object obj = loader.newInstance(classArg, Object.class, "search.similarities.");
       if (obj instanceof SimilarityFactory) {
         // configure a factory, get a similarity back
diff --git a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
index e7a748c..0dd71f8 100644
--- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
@@ -70,6 +70,8 @@ import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.rest.schema.FieldTypeXmlAdapter;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.DOMConfigNode;
 import org.apache.solr.util.FileUtils;
 import org.apache.solr.util.RTimer;
 import org.apache.zookeeper.CreateMode;
@@ -1308,7 +1310,7 @@ public final class ManagedIndexSchema extends IndexSchema {
     Map<String,FieldType> newFieldTypes = new HashMap<>();
     List<SchemaAware> schemaAwareList = new ArrayList<>();
     FieldTypePluginLoader typeLoader = new FieldTypePluginLoader(this, newFieldTypes, schemaAwareList);
-    typeLoader.loadSingle(solrClassLoader, FieldTypeXmlAdapter.toNode(options));
+    typeLoader.loadSingle(solrClassLoader, new DOMConfigNode(FieldTypeXmlAdapter.toNode(options)));
     FieldType ft = newFieldTypes.get(typeName);
     if (!schemaAwareList.isEmpty())
       schemaAware.addAll(schemaAwareList);
diff --git a/solr/core/src/java/org/apache/solr/util/DOMConfigNode.java b/solr/core/src/java/org/apache/solr/util/DOMConfigNode.java
new file mode 100644
index 0000000..1cccbce
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/DOMConfigNode.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+import org.apache.solr.cluster.api.SimpleMap;
+import org.apache.solr.common.ConfigNode;
+import org.apache.solr.common.util.WrappedSimpleMap;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+/**
+ * Read using DOM
+ */
+public class DOMConfigNode implements ConfigNode {
+
+  private final Node node;
+  SimpleMap<String> attrs;
+
+  @Override
+  public String name() {
+    return node.getNodeName();
+  }
+
+  @Override
+  public String textValue() {
+    return DOMUtil.getText(node);
+  }
+
+  public DOMConfigNode(Node node) {
+    this.node = node;
+  }
+
+  @Override
+  public SimpleMap<String> attributes() {
+    if (attrs != null) return attrs;
+    return attrs = new WrappedSimpleMap<>(DOMUtil.toMap(node.getAttributes()));
+  }
+
+  @Override
+  public ConfigNode child(String name) {
+    Node n  =  DOMUtil.getChild(node, name);
+    return n == null? null: new DOMConfigNode(n);
+  }
+
+  @Override
+  public List<ConfigNode> children(String name) {
+    List<ConfigNode> result = new ArrayList<>();
+    forEachChild(it -> {
+      if (name.equals(it.name())) {
+        result.add(it);
+      }
+      return Boolean.TRUE;
+    });
+    return result;
+  }
+
+  @Override
+  public void forEachChild(Function<ConfigNode, Boolean> fun) {
+    NodeList nlst = node.getChildNodes();
+    for (int i = 0; i < nlst.getLength(); i++) {
+      Node item = nlst.item(i);
+      if(item.getNodeType() != Node.ELEMENT_NODE) continue;
+      Boolean toContinue = fun.apply(new DOMConfigNode(item));
+      if (Boolean.FALSE == toContinue) break;
+    }
+  }
+}
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 d042531..48f07d0 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
@@ -20,7 +20,9 @@ import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
+import java.util.function.Consumer;
 
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.SolrClassLoader;
@@ -86,7 +88,7 @@ public abstract class AbstractPluginLoader<T>
    * @param node - the XML node defining this plugin
    */
   @SuppressWarnings("unchecked")
-  protected T create(SolrClassLoader loader, String name, String className, Node node ) throws Exception
+  protected T create(SolrClassLoader loader, String name, String className, ConfigNode node ) throws Exception
   {
     return loader.newInstance(className, pluginClassType, getDefaultPackages());
   }
@@ -103,7 +105,7 @@ public abstract class AbstractPluginLoader<T>
    * @param plugin - the plugin to initialize
    * @param node - the XML node defining this plugin
    */
-  abstract protected void init( T plugin, Node node ) throws Exception;
+  abstract protected void init( T plugin, ConfigNode node ) throws Exception;
 
   /**
    * Initializes and registers each plugin in the list.
@@ -135,15 +137,13 @@ public abstract class AbstractPluginLoader<T>
    * If a default element is defined, it will be returned from this function.
    * 
    */
-  public T load(SolrClassLoader loader, NodeList nodes )
+  public T load(SolrClassLoader loader, List<ConfigNode> nodes )
   {
     List<PluginInitInfo> info = new ArrayList<>();
     T defaultPlugin = null;
     
     if (nodes !=null ) {
-      for (int i=0; i<nodes.getLength(); i++) {
-        Node node = nodes.item(i);
-  
+      for (ConfigNode node : nodes) {
         String name = null;
         try {
           name = DOMUtil.getAttr(node, NAME, requireName ? type : null);
@@ -225,7 +225,7 @@ public abstract class AbstractPluginLoader<T>
    * The created class for the plugin will be returned from this function.
    * 
    */
-  public T loadSingle(SolrClassLoader loader, Node node) {
+  public T loadSingle(SolrClassLoader loader, ConfigNode node) {
     List<PluginInitInfo> info = new ArrayList<>();
     T plugin = null;
 
@@ -277,9 +277,9 @@ public abstract class AbstractPluginLoader<T>
    */
   private class PluginInitInfo {
     final T plugin;
-    final Node node;
+    final ConfigNode node;
 
-    PluginInitInfo(T plugin, Node node) {
+    PluginInitInfo(T plugin, ConfigNode node) {
       this.plugin = plugin;
       this.node = node;
     }
diff --git a/solr/core/src/java/org/apache/solr/util/plugin/MapPluginLoader.java b/solr/core/src/java/org/apache/solr/util/plugin/MapPluginLoader.java
index 9cf68f6..46bd79f 100644
--- a/solr/core/src/java/org/apache/solr/util/plugin/MapPluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/util/plugin/MapPluginLoader.java
@@ -19,7 +19,7 @@ package org.apache.solr.util.plugin;
 import java.util.Map;
 
 import org.apache.solr.common.util.DOMUtil;
-import org.w3c.dom.Node;
+import org.apache.solr.common.ConfigNode;
 
 import static org.apache.solr.common.params.CommonParams.NAME;
 
@@ -38,8 +38,8 @@ public class MapPluginLoader<T extends MapInitializedPlugin> extends AbstractPlu
   }
 
   @Override
-  protected void init(T plugin, Node node) throws Exception {
-    Map<String, String> params = DOMUtil.toMapExcept(node.getAttributes(), NAME, "class");
+  protected void init(T plugin, ConfigNode node) throws Exception {
+    Map<String, String> params = DOMUtil.toMapExcept(node, NAME, "class");
     plugin.init( params );
   }
 
diff --git a/solr/core/src/java/org/apache/solr/util/plugin/NamedListPluginLoader.java b/solr/core/src/java/org/apache/solr/util/plugin/NamedListPluginLoader.java
index ae202db..5736abf 100644
--- a/solr/core/src/java/org/apache/solr/util/plugin/NamedListPluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/util/plugin/NamedListPluginLoader.java
@@ -19,7 +19,7 @@ package org.apache.solr.util.plugin;
 import java.util.Map;
 
 import org.apache.solr.common.util.DOMUtil;
-import org.w3c.dom.Node;
+import org.apache.solr.common.ConfigNode;
 
 /**
  *
@@ -35,7 +35,7 @@ public class NamedListPluginLoader<T extends NamedListInitializedPlugin> extends
   }
 
   @Override
-  protected void init(T plugin,Node node) throws Exception {
+  protected void init(T plugin, ConfigNode node) throws Exception {
     plugin.init( DOMUtil.childNodesToNamedList(node) );
   }
 
diff --git a/solr/solrj/src/java/org/apache/solr/cluster/api/SimpleMap.java b/solr/solrj/src/java/org/apache/solr/cluster/api/SimpleMap.java
index ca747b9..81da171 100644
--- a/solr/solrj/src/java/org/apache/solr/cluster/api/SimpleMap.java
+++ b/solr/solrj/src/java/org/apache/solr/cluster/api/SimpleMap.java
@@ -35,6 +35,11 @@ public interface SimpleMap<T> extends MapWriter {
   /**get a value by key. If not present , null is returned */
   T get(String key);
 
+  default T get(String key, T def) {
+    T val = get(key);
+    return val == null ? def : val;
+  }
+
   /**Navigate through all keys and values */
   void forEachEntry(BiConsumer<String, ? super T> fun);
 
diff --git a/solr/solrj/src/java/org/apache/solr/common/ConfigNode.java b/solr/solrj/src/java/org/apache/solr/common/ConfigNode.java
new file mode 100644
index 0000000..90b0b81
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/ConfigNode.java
@@ -0,0 +1,73 @@
+/*
+ * 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.common;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import org.apache.solr.cluster.api.SimpleMap;
+
+/**
+ * A generic interface that represents a config file, mostly XML
+ */
+public interface ConfigNode {
+
+  /**Name of the tag
+   */
+  String name();
+
+  /**Text value of the node
+   */
+  String textValue();
+
+  /**Attributes
+   */
+  SimpleMap<String> attributes();
+
+  /** Child by name
+   */
+  ConfigNode child(String name);
+
+  default ConfigNode child(Predicate<ConfigNode> test) {
+    ConfigNode[] result = new ConfigNode[1];
+    forEachChild(it -> {
+      if (test.test(it)) {
+        result[0] = it;
+        return Boolean.FALSE;
+      }
+      return Boolean.TRUE;
+    });
+    return result[0];
+  }
+
+  default List<ConfigNode> children(Predicate<ConfigNode> test) {
+    List<ConfigNode> result = new ArrayList<>();
+    forEachChild(it -> {
+      if (test.test(it)) result.add(it);
+      return Boolean.TRUE;
+    });
+    return result;
+  }
+
+  List<ConfigNode> children(String name);
+
+  void forEachChild(Function<ConfigNode, Boolean> fun);
+
+
+}
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/DOMUtil.java b/solr/solrj/src/java/org/apache/solr/common/util/DOMUtil.java
index c86fcea..206d84b 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/DOMUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/DOMUtil.java
@@ -22,7 +22,10 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 
+import com.google.common.collect.ImmutableSet;
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.w3c.dom.NamedNodeMap;
 import org.w3c.dom.Node;
@@ -37,9 +40,23 @@ public class DOMUtil {
 
   public static final String XML_RESERVED_PREFIX = "xml";
 
+  static final Set<String>  NL_TAGS = ImmutableSet.of("str", "int","long","float","double","bool");
+
+
   public static Map<String,String> toMap(NamedNodeMap attrs) {
     return toMapExcept(attrs);
   }
+  public static Map<String,String> toMap(ConfigNode node) {
+    return toMapExcept(node);
+  }
+  public static Map<String,String> toMapExcept(ConfigNode node, String... exclusions) {
+    Map<String,String> args = new HashMap<>();
+    node.attributes().forEachEntry((k, v) -> {
+      for (String ex : exclusions) if (ex.equals(k)) return;
+        args.put(k,v);
+    });
+    return args;
+  }
 
   public static Map<String,String> toMapExcept(NamedNodeMap attrs, String... exclusions) {
     Map<String,String> args = new HashMap<>();
@@ -92,6 +109,16 @@ public class DOMUtil {
     return val;
   }
 
+  public static String getAttr(ConfigNode node, String name, String missing_err) {
+    String attr = node.attributes().get(name);
+    if (attr == null) {
+      if (missing_err == null) return null;
+      throw new RuntimeException(missing_err + ": missing mandatory attribute '" + name + "'");
+    }
+    return attr;
+
+  }
+
   public static String getAttr(Node node, String name, String missing_err) {
     return getAttr(node.getAttributes(), name, missing_err);
   }
@@ -110,6 +137,10 @@ public class DOMUtil {
     return nodesToList(nd.getChildNodes());
   }
 
+  public static NamedList<Object> childNodesToNamedList(ConfigNode node) {
+    return readNamedListChildren(node);
+  }
+
   public static NamedList<Object> nodesToNamedList(NodeList nlst) {
     NamedList<Object> clst = new NamedList<>();
     for (int i=0; i<nlst.getLength(); i++) {
@@ -160,35 +191,66 @@ public class DOMUtil {
       val = childNodesToList(nd);
     } else {
       final String textValue = getText(nd);
-      try {
-        if ("str".equals(type)) {
-          val = textValue;
-        } else if ("int".equals(type)) {
-          val = Integer.valueOf(textValue);
-        } else if ("long".equals(type)) {
-          val = Long.valueOf(textValue);
-        } else if ("float".equals(type)) {
-          val = Float.valueOf(textValue);
-        } else if ("double".equals(type)) {
-          val = Double.valueOf(textValue);
-        } else if ("bool".equals(type)) {
-          val = StrUtils.parseBool(textValue);
-        }
-        // :NOTE: Unexpected Node names are ignored
-        // :TODO: should we generate an error here?
-      } catch (NumberFormatException nfe) {
-        throw new SolrException
-          (SolrException.ErrorCode.SERVER_ERROR,
-           "Value " + (null != name ? ("of '" +name+ "' ") : "") +
-           "can not be parsed as '" +type+ "': \"" + textValue + "\"",
-           nfe);
-      }
+      val = parseVal(type, name, textValue);
     }
 
     if (nlst != null) nlst.add(name,val);
     if (arr != null) arr.add(val);
   }
 
+  private static Object parseVal(String type, String name, String textValue) {
+    Object val = null;
+    try {
+      if ("str".equals(type)) {
+        val = textValue;
+      } else if ("int".equals(type)) {
+        val = Integer.valueOf(textValue);
+      } else if ("long".equals(type)) {
+        val = Long.valueOf(textValue);
+      } else if ("float".equals(type)) {
+        val = Float.valueOf(textValue);
+      } else if ("double".equals(type)) {
+        val = Double.valueOf(textValue);
+      } else if ("bool".equals(type)) {
+        val = StrUtils.parseBool(textValue);
+      }
+      // :NOTE: Unexpected Node names are ignored
+      // :TODO: should we generate an error here?
+    } catch (NumberFormatException nfe) {
+      throw new SolrException
+        (SolrException.ErrorCode.SERVER_ERROR,
+         "Value " + (null != name ? ("of '" + name + "' ") : "") +
+         "can not be parsed as '" + type + "': \"" + textValue + "\"",
+         nfe);
+    }
+    return val;
+  }
+
+  public static NamedList<Object> readNamedListChildren(ConfigNode configNode) {
+    NamedList<Object> result = new NamedList<>();
+    configNode.forEachChild(it -> {
+      String tag = it.name();
+      String varName = it.attributes().get("name");
+      if (NL_TAGS.contains(tag)) {
+        result.add(varName, parseVal(tag, varName, it.textValue()));
+      }
+      if ("lst".equals(tag)) {
+        result.add(varName, readNamedListChildren(it));
+      } else if ("arr".equals(tag)) {
+        List<Object> l = new ArrayList<>();
+        result.add(varName, l);
+        it.forEachChild(n -> {
+          if (NL_TAGS.contains(n.name())) {
+            l.add(parseVal(n.name(), null, n.textValue()));
+          }
+          return Boolean.TRUE;
+        });
+      }
+      return Boolean.TRUE;
+    });
+    return result;
+  }
+
   /**
    * Drop in replacement for Node.getTextContent().
    *