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

[lucene-solr] 08/18: cache and reuse

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 a73c2c526782cd6d0e4ccbe1eb4cc139ecd1e052
Author: noblepaul <no...@gmail.com>
AuthorDate: Thu Sep 10 17:54:36 2020 +1000

    cache and reuse
---
 .../apache/solr/cloud/CloudConfigSetService.java   | 30 +++++++++++-
 .../apache/solr/cloud/ZkSolrResourceLoader.java    |  7 +--
 .../org/apache/solr/core/ConfigSetService.java     | 18 ++++++-
 .../java/org/apache/solr/schema/IndexSchema.java   | 25 ++++++----
 .../org/apache/solr/schema/IndexSchemaFactory.java | 55 ++++++++++++++++++----
 .../org/apache/solr/schema/ManagedIndexSchema.java |  5 +-
 .../solr/schema/ManagedIndexSchemaFactory.java     |  7 +--
 .../java/org/apache/solr/schema/SchemaManager.java |  3 +-
 .../apache/solr/schema/ZkIndexSchemaReader.java    |  2 +-
 .../java/org/apache/solr/util/DataConfigNode.java  | 24 +++++-----
 .../solr/schema/TestManagedSchemaThreadSafety.java |  8 ++--
 .../java/org/apache/solr/common/ConfigNode.java    |  1 +
 12 files changed, 138 insertions(+), 47 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java b/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
index 644b49e..54c515e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
@@ -17,8 +17,12 @@
 package org.apache.solr.cloud;
 
 import java.lang.invoke.MethodHandles;
+import java.lang.ref.WeakReference;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.solr.cloud.api.collections.CreateCollectionCmd;
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ZkConfigManager;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -39,7 +43,7 @@ import org.slf4j.LoggerFactory;
  */
 public class CloudConfigSetService extends ConfigSetService {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  
+  private Map<String, ConfigCacheEntry> cache = new ConcurrentHashMap<>();
   private final ZkController zkController;
 
   public CloudConfigSetService(SolrResourceLoader loader, boolean shareSchema, ZkController zkController) {
@@ -47,6 +51,20 @@ public class CloudConfigSetService extends ConfigSetService {
     this.zkController = zkController;
   }
 
+  public void storeConfig(String resource, ConfigNode config, int znodeVersion) {
+    cache.put(resource, new ConfigCacheEntry(config, znodeVersion));
+  }
+
+  public ConfigNode getConfig(String resource, int znodeVersion) {
+    ConfigCacheEntry e = cache.get(resource);
+    if (e == null) return null;
+    ConfigNode configNode = e.configNode.get();
+    if (configNode == null) cache.remove(resource);
+    if (e.znodeVersion == znodeVersion) return configNode;
+    if (e.znodeVersion < znodeVersion) cache.remove(resource);
+    return null;
+  }
+
   @Override
   public SolrResourceLoader createCoreResourceLoader(CoreDescriptor cd) {
     final String colName = cd.getCollectionName();
@@ -111,4 +129,14 @@ public class CloudConfigSetService extends ConfigSetService {
   public String configSetName(CoreDescriptor cd) {
     return "configset " + cd.getConfigSet();
   }
+
+  private static class ConfigCacheEntry {
+    final WeakReference<ConfigNode> configNode;
+    final int znodeVersion;
+
+    private ConfigCacheEntry(ConfigNode configNode, int znodeVersion) {
+      this.configNode = new WeakReference<>(configNode);
+      this.znodeVersion = znodeVersion;
+    }
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
index 0e17b5c..b52d6bb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
@@ -81,7 +81,7 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
         if (zkController.pathExists(file)) {
           Stat stat = new Stat();
           byte[] bytes = zkController.getZkClient().getData(file, null, stat, true);
-          return new ZkByteArrayInputStream(bytes, stat);
+          return new ZkByteArrayInputStream(bytes, file, stat);
         } else {
           //Path does not exists. We only retry for session expired exceptions.
           break;
@@ -126,11 +126,12 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
 
   public static class ZkByteArrayInputStream extends ByteArrayInputStream{
 
+    public final String fileName;
     private final Stat stat;
-    public ZkByteArrayInputStream(byte[] buf, Stat stat) {
+    public ZkByteArrayInputStream(byte[] buf, String fileName, Stat stat) {
       super(buf);
+      this.fileName = fileName;
       this.stat = stat;
-
     }
 
     public Stat getStat(){
diff --git a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
index 1d45bc5..691f17a 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
@@ -28,12 +28,14 @@ import com.github.benmanes.caffeine.cache.Caffeine;
 import org.apache.solr.cloud.CloudConfigSetService;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.IndexSchemaFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.xml.sax.InputSource;
 
 /**
  * Service class used by the CoreContainer to load ConfigSets for use in SolrCore
@@ -135,14 +137,14 @@ public abstract class ConfigSetService {
         // note: luceneMatchVersion influences the schema
         String cacheKey = configSet + "/" + guessSchemaName + "/" + modVersion + "/" + solrConfig.luceneMatchVersion;
         return schemaCache.get(cacheKey,
-            (key) -> indexSchemaFactory.create(cdSchemaName, solrConfig));
+            (key) -> indexSchemaFactory.create(cdSchemaName, solrConfig, ConfigSetService.this));
       } else {
         log.warn("Unable to get schema modification version, configSet={} schema={}", configSet, guessSchemaName);
         // see explanation above; "guessSchema" is a guess
       }
     }
 
-    return indexSchemaFactory.create(cdSchemaName, solrConfig);
+    return indexSchemaFactory.create(cdSchemaName, solrConfig, this);
   }
 
   /**
@@ -186,6 +188,18 @@ public abstract class ConfigSetService {
    */
   public abstract String configSetName(CoreDescriptor cd);
 
+  public interface ConfigResource {
+
+    default String resourceName() {return null;}
+
+    InputSource getSource();
+
+    default ConfigNode getParsed() {return null;}
+
+    default void storeParsed(ConfigNode node){}
+
+  }
+
   /**
    * The Solr standalone version of ConfigSetService.
    *
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 772e049..20281e3 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -64,6 +64,7 @@ import org.apache.solr.common.util.DOMUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.core.ConfigSetService;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.core.XmlConfigFile;
@@ -80,7 +81,6 @@ import org.apache.solr.util.PayloadUtils;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.xml.sax.InputSource;
 
 import static java.util.Arrays.asList;
 import static java.util.Collections.singletonList;
@@ -169,6 +169,7 @@ public class IndexSchema {
    * directives that target them.
    */
   protected Map<SchemaField, Integer> copyFieldTargetCounts = new HashMap<>();
+  private ConfigNode rootNode;
 //  static AtomicLong totalSchemaLoadTime = new AtomicLong();
 
 
@@ -177,17 +178,20 @@ public class IndexSchema {
    * By default, this follows the normal config path directory searching rules.
    * @see SolrResourceLoader#openResource
    */
-  public IndexSchema(String name, InputSource is, Version luceneVersion, SolrResourceLoader resourceLoader, Properties substitutableProperties) {
+  public IndexSchema(String name, ConfigSetService.ConfigResource schemaResource, Version luceneVersion, SolrResourceLoader resourceLoader, Properties substitutableProperties) {
     this(luceneVersion, resourceLoader, substitutableProperties);
 
     this.resourceName = Objects.requireNonNull(name);
+    ConfigNode.SUBSTITUTES.set(substitutableProperties::getProperty);
     try {
 //      long start = System.currentTimeMillis();
-      readSchema(is);
+      readSchema(schemaResource);
 //      System.out.println("schema-load-time : "+ totalSchemaLoadTime.addAndGet (System.currentTimeMillis() - start));
       loader.inform(loader);
     } catch (IOException e) {
       throw new RuntimeException(e);
+    } finally {
+      ConfigNode.SUBSTITUTES.remove();
     }
   }
 
@@ -478,18 +482,21 @@ public class IndexSchema {
     }
   }
 
-  protected void readSchema(InputSource is) {
+  protected void readSchema(ConfigSetService.ConfigResource is) {
     assert null != is : "schema InputSource should never be null";
     try {
-      // 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, null);
+      rootNode = is.getParsed();
+      if(rootNode == null) {
+        // 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.getSource(), SLASH+SCHEMA+SLASH, null);
 //      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 = new DataConfigNode(new DOMConfigNode(schemaConf.getDocument().getDocumentElement()),
-          substitutableProperties::getProperty) ;
+        rootNode = new DataConfigNode(new DOMConfigNode(schemaConf.getDocument().getDocumentElement())) ;
+        is.storeParsed(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"
diff --git a/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java b/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java
index 095efd4..0391aa2 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java
@@ -19,8 +19,12 @@ package org.apache.solr.schema;
 import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
 
+import org.apache.solr.cloud.CloudConfigSetService;
+import org.apache.solr.cloud.ZkSolrResourceLoader;
+import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.core.ConfigSetService;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrResourceLoader;
@@ -33,10 +37,12 @@ import org.xml.sax.InputSource;
 /** Base class for factories for IndexSchema implementations */
 public abstract class IndexSchemaFactory implements NamedListInitializedPlugin {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  /** Instantiates the configured schema factory, then calls create on it. */
   public static IndexSchema buildIndexSchema(String resourceName, SolrConfig config) {
-    return newIndexSchemaFactory(config).create(resourceName, config);
+    return buildIndexSchema(resourceName, config, null);
+  }
+  /** Instantiates the configured schema factory, then calls create on it. */
+  public static IndexSchema buildIndexSchema(String resourceName, SolrConfig config, ConfigSetService configSetService) {
+    return newIndexSchemaFactory(config).create(resourceName, config, configSetService);
   }
 
   /** Instantiates us from {@link SolrConfig}. */
@@ -54,7 +60,7 @@ public abstract class IndexSchemaFactory implements NamedListInitializedPlugin {
 
   /**
    * Returns the resource (file) name that will be used for the schema itself.  The answer may be a guess.
-   * Do not pass the result of this to {@link #create(String, SolrConfig)}.
+   * Do not pass the result of this to {@link #create(String, SolrConfig, ConfigSetService)}.
    * The input is the name coming from the {@link org.apache.solr.core.CoreDescriptor}
    * which acts as a default or asked-for name.
    */
@@ -65,7 +71,7 @@ public abstract class IndexSchemaFactory implements NamedListInitializedPlugin {
   /**
    * Returns an index schema created from a local resource.  The input is usually from the core descriptor.
    */
-  public IndexSchema create(String resourceName, SolrConfig config) {
+  public IndexSchema create(String resourceName, SolrConfig config, ConfigSetService configSetService) {
     SolrResourceLoader loader = config.getResourceLoader();
     InputStream schemaInputStream = null;
 
@@ -80,10 +86,41 @@ public abstract class IndexSchemaFactory implements NamedListInitializedPlugin {
       log.error(msg, e);
       throw new SolrException(ErrorCode.SERVER_ERROR, msg, e);
     }
-    InputSource inputSource = new InputSource(schemaInputStream);
-    inputSource.setSystemId(SystemIdResolver.createSystemIdFromResourceName(resourceName));
-    IndexSchema schema = new IndexSchema(resourceName, inputSource, config.luceneMatchVersion, loader, config.getSubstituteProperties());
-    return schema;
+    InputStream is = schemaInputStream;
+    String name = resourceName;
+    ConfigSetService.ConfigResource schemaResource  = new ConfigSetService.ConfigResource() {
+      @Override
+      public InputSource getSource() {
+        InputSource inputSource = new InputSource(is);
+        inputSource.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
+        return inputSource;
+      }
+
+      @Override
+      public String resourceName() {
+        return name;
+      }
+
+      @Override
+      public ConfigNode getParsed() {
+        if (configSetService instanceof CloudConfigSetService && is instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
+          CloudConfigSetService cloudConfigSetService = (CloudConfigSetService) configSetService;
+          ZkSolrResourceLoader.ZkByteArrayInputStream zkis = (ZkSolrResourceLoader.ZkByteArrayInputStream) is;
+          return cloudConfigSetService.getConfig(zkis.fileName, zkis.getStat().getVersion());
+        }
+        return null;
+      }
+
+      @Override
+      public void storeParsed(ConfigNode node) {
+        if (configSetService instanceof CloudConfigSetService && is instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
+          CloudConfigSetService cloudConfigSetService = (CloudConfigSetService) configSetService;
+          ZkSolrResourceLoader.ZkByteArrayInputStream zkis = (ZkSolrResourceLoader.ZkByteArrayInputStream) is;
+          cloudConfigSetService.storeConfig(zkis.fileName, node, zkis.getStat().getVersion());
+        }
+      }
+    };
+    return new IndexSchema(resourceName, schemaResource, config.luceneMatchVersion, loader, config.getSubstituteProperties());
   }
 
 }
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 0dd71f8..dd17c5b 100644
--- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
@@ -67,10 +67,10 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.ConfigSetService;
 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;
@@ -79,7 +79,6 @@ import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.xml.sax.InputSource;
 
 import static org.apache.solr.core.SolrResourceLoader.informAware;
 
@@ -104,7 +103,7 @@ public final class ManagedIndexSchema extends IndexSchema {
    * By default, this follows the normal config path directory searching rules.
    * @see org.apache.solr.core.SolrResourceLoader#openResource
    */
-  ManagedIndexSchema(SolrConfig solrConfig, String name, InputSource is, boolean isMutable,
+  ManagedIndexSchema(SolrConfig solrConfig, String name, ConfigSetService.ConfigResource is, boolean isMutable,
                      String managedSchemaResourceName, int schemaZkVersion, Object schemaUpdateLock) {
     super(name, is, solrConfig.luceneMatchVersion, solrConfig.getResourceLoader(), solrConfig.getSubstituteProperties());
     this.isMutable = isMutable;
diff --git a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
index ef5df42..a7e6aef 100644
--- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
+++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
@@ -30,6 +30,7 @@ import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkCmdExecutor;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.ConfigSetService;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
@@ -109,7 +110,7 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
    * renamed by appending the extension named in {@link #UPGRADED_SCHEMA_EXTENSION}.
    */
   @Override
-  public ManagedIndexSchema create(String resourceName, SolrConfig config) {
+  public ManagedIndexSchema create(String resourceName, SolrConfig config, ConfigSetService configSetService) {
     this.resourceName = resourceName;
     this.config = config;
     this.loader = config.getResourceLoader();
@@ -174,8 +175,8 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
     }
     InputSource inputSource = new InputSource(schemaInputStream);
     inputSource.setSystemId(SystemIdResolver.createSystemIdFromResourceName(loadedResource));
-    schema = new ManagedIndexSchema(config, loadedResource, inputSource, isMutable,
-                                    managedSchemaResourceName, schemaZkVersion, getSchemaUpdateLock());
+    schema = new ManagedIndexSchema(config, loadedResource, () -> inputSource, isMutable,
+        managedSchemaResourceName, schemaZkVersion, getSchemaUpdateLock());
     if (shouldUpgrade) {
       // Persist the managed schema if it doesn't already exist
       synchronized (schema.getSchemaUpdateLock()) {
diff --git a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
index f539969..be126f2 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
@@ -451,10 +451,11 @@ public class SchemaManager {
         return (ManagedIndexSchema) core.getLatestSchema();
       }
       InputStream in = resourceLoader.openResource(name);
+      InputSource is = new InputSource(in);
       if (in instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
         int version = ((ZkSolrResourceLoader.ZkByteArrayInputStream) in).getStat().getVersion();
         log.info("managed schema loaded . version : {} ", version);
-        return new ManagedIndexSchema(core.getSolrConfig(), name, new InputSource(in), true, name, version,
+        return new ManagedIndexSchema(core.getSolrConfig(), name, () -> is, true, name, version,
             core.getLatestSchema().getSchemaUpdateLock());
       } else {
         return (ManagedIndexSchema) core.getLatestSchema();
diff --git a/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java b/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java
index 3b867ce..56daf01 100644
--- a/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java
+++ b/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java
@@ -174,7 +174,7 @@ public class ZkIndexSchemaReader implements OnReconnect {
           InputSource inputSource = new InputSource(new ByteArrayInputStream(data));
           String resourceName = managedIndexSchemaFactory.getManagedSchemaResourceName();
           ManagedIndexSchema newSchema = new ManagedIndexSchema
-              (managedIndexSchemaFactory.getConfig(), resourceName, inputSource, managedIndexSchemaFactory.isMutable(), 
+              (managedIndexSchemaFactory.getConfig(), resourceName, () -> inputSource, managedIndexSchemaFactory.isMutable(),
                   resourceName, stat.getVersion(), oldSchema.getSchemaUpdateLock());
           managedIndexSchemaFactory.setSchema(newSchema);
           long stop = System.nanoTime();
diff --git a/solr/core/src/java/org/apache/solr/util/DataConfigNode.java b/solr/core/src/java/org/apache/solr/util/DataConfigNode.java
index 3867af7..9c825e8 100644
--- a/solr/core/src/java/org/apache/solr/util/DataConfigNode.java
+++ b/solr/core/src/java/org/apache/solr/util/DataConfigNode.java
@@ -37,36 +37,38 @@ public class DataConfigNode implements ConfigNode {
   final String name;
   final SimpleMap<String> attributes;
   private final Map<String, List<ConfigNode>> kids = new HashMap<>();
-  private String textData;
-  private final Function<String, String> propertySubstitution;
+  private final String textData;
 
-  public DataConfigNode(ConfigNode root, Function<String, String> propertySubstitution) {
-    this.propertySubstitution = propertySubstitution;
+  public DataConfigNode(ConfigNode root) {
     name = root.name();
     attributes = wrap(root.attributes());
     textData = root.textValue();
-    if (textData != null) textData = PropertiesUtil.substitute(textData.trim(), propertySubstitution);
     root.forEachChild(it -> {
       List<ConfigNode> nodes = kids.computeIfAbsent(it.name(),
           k -> new ArrayList<>());
 
-     nodes.add(new DataConfigNode(it,  propertySubstitution));
+     nodes.add(new DataConfigNode(it));
       return Boolean.TRUE;
     });
 
   }
 
+  public String subtituteVal(String s) {
+    Function<String, String> props = SUBSTITUTES.get();
+    if (props == null) return s;
+    return PropertiesUtil.substitute(s, props);
+  }
+
   private SimpleMap<String> wrap(SimpleMap<String> delegate) {
-    return propertySubstitution == null ? delegate :
-        new SimpleMap<>() {
+    return new SimpleMap<>() {
           @Override
           public String get(String key) {
-            return PropertiesUtil.substitute(delegate.get(key), propertySubstitution);
+            return subtituteVal(delegate.get(key));
           }
 
           @Override
           public void forEachEntry(BiConsumer<String, ? super String> fun) {
-            delegate.forEachEntry((k, v) -> fun.accept(k, PropertiesUtil.substitute(v, propertySubstitution)));
+            delegate.forEachEntry((k, v) -> fun.accept(k, subtituteVal(v)));
           }
 
           @Override
@@ -83,7 +85,7 @@ public class DataConfigNode implements ConfigNode {
 
   @Override
   public String textValue() {
-    return textData;
+    return  subtituteVal(textData);
   }
 
   @Override
diff --git a/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java b/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java
index c611359..e2d8737 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java
@@ -17,9 +17,6 @@
 
 package org.apache.solr.schema;
 
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.List;
@@ -50,6 +47,9 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 public class TestManagedSchemaThreadSafety extends SolrTestCaseJ4 {
 
   private static final class SuspendingZkClient extends SolrZkClient {
@@ -180,7 +180,7 @@ public class TestManagedSchemaThreadSafety extends SolrTestCaseJ4 {
 
         ManagedIndexSchemaFactory factory = new ManagedIndexSchemaFactory();
         factory.init(new NamedList());
-        factory.create("schema.xml", solrConfig);
+        factory.create("schema.xml", solrConfig, null);
       }
       catch (Exception e) {
         throw new RuntimeException(e);
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 f046440..0f6a813 100644
--- a/solr/solrj/src/java/org/apache/solr/common/ConfigNode.java
+++ b/solr/solrj/src/java/org/apache/solr/common/ConfigNode.java
@@ -29,6 +29,7 @@ import org.apache.solr.cluster.api.SimpleMap;
  * A generic interface that represents a config file, mostly XML
  */
 public interface ConfigNode {
+  ThreadLocal<Function<String,String>> SUBSTITUTES = new ThreadLocal<>();
 
   /**
    * Name of the tag