You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2013/05/30 09:53:46 UTC

svn commit: r1487777 [36/50] - in /lucene/dev/branches/security: ./ dev-tools/ dev-tools/eclipse/dot.settings/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/replicator/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/ma...

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/core/Config.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/core/Config.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/core/Config.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/core/Config.java Thu May 30 07:53:18 2013
@@ -25,7 +25,10 @@ import org.apache.solr.common.util.XMLEr
 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 org.xml.sax.SAXException;
 import org.apache.commons.io.IOUtils;
@@ -34,13 +37,25 @@ import javax.xml.namespace.QName;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.Transformer;
 import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMResult;
+import javax.xml.transform.dom.DOMSource;
 import javax.xml.xpath.XPath;
 import javax.xml.xpath.XPathConstants;
 import javax.xml.xpath.XPathExpressionException;
 import javax.xml.xpath.XPathFactory;
+
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
@@ -65,18 +80,6 @@ public class Config {
     this( loader, name, null, null );
   }
 
-  /**
-   * For the transition from using solr.xml to solr.properties, see SOLR-4196. Remove
-   * for 5.0, thus it's already deprecated
-   * @param loader - Solr resource loader
-   * @param cfg    - SolrConfig, for backwards compatability with solr.xml layer.
-   * @throws TransformerException if the XML file is mal-formed
-   */
-  @Deprecated
-  public Config(SolrResourceLoader loader, Config cfg) throws TransformerException {
-    this(loader, null, ConfigSolrXmlBackCompat.copyDoc(cfg.getDocument()));
-  }
-
   public Config(SolrResourceLoader loader, String name, InputSource is, String prefix) throws ParserConfigurationException, IOException, SAXException 
   {
     this(loader, name, is, prefix, true);
@@ -154,6 +157,16 @@ public class Config {
     this.loader = loader;
   }
 
+  
+  private static Document copyDoc(Document doc) throws TransformerException {
+    TransformerFactory tfactory = TransformerFactory.newInstance();
+    Transformer tx = tfactory.newTransformer();
+    DOMSource source = new DOMSource(doc);
+    DOMResult result = new DOMResult();
+    tx.transform(source, result);
+    return (Document) result.getNode();
+  }
+  
   /**
    * @since solr 1.3
    */
@@ -235,6 +248,98 @@ public class Config {
     }
   }
 
+  public NodeList getNodeList(String path, boolean errIfMissing) {
+    XPath xpath = xpathFactory.newXPath();
+    String xstr = normalize(path);
+
+    try {
+      NodeList nodeList = (NodeList)xpath.evaluate(xstr, doc, XPathConstants.NODESET);
+
+      if (null == nodeList) {
+        if (errIfMissing) {
+          throw new RuntimeException(name + " missing "+path);
+        } else {
+          log.debug(name + " missing optional " + path);
+          return null;
+        }
+      }
+
+      log.trace(name + ":" + path + "=" + nodeList);
+      return nodeList;
+
+    } catch (XPathExpressionException e) {
+      SolrException.log(log,"Error in xpath",e);
+      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error in xpath:" + xstr + " for " + name,e);
+    } catch (SolrException e) {
+      throw(e);
+    } catch (Throwable e) {
+      SolrException.log(log,"Error in xpath",e);
+      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error in xpath:" + xstr+ " for " + name,e);
+    }
+  }
+
+  /**
+   * Returns the set of attributes on the given element that are not among the given knownAttributes,
+   * or null if all attributes are known.
+   */
+  public Set<String> getUnknownAttributes(Element element, String... knownAttributes) {
+    Set<String> knownAttributeSet = new HashSet<String>(Arrays.asList(knownAttributes));
+    Set<String> unknownAttributeSet = null;
+    NamedNodeMap attributes = element.getAttributes();
+    for (int i = 0 ; i < attributes.getLength() ; ++i) {
+      final String attributeName = attributes.item(i).getNodeName();
+      if ( ! knownAttributeSet.contains(attributeName)) {
+        if (null == unknownAttributeSet) {
+          unknownAttributeSet = new HashSet<String>();
+        }
+        unknownAttributeSet.add(attributeName);
+      }
+    }
+    return unknownAttributeSet;
+  }
+
+  /**
+   * Logs an error and throws an exception if any of the element(s) at the given elementXpath
+   * contains an attribute name that is not among knownAttributes. 
+   */
+  public void complainAboutUnknownAttributes(String elementXpath, String... knownAttributes) {
+    SortedMap<String,SortedSet<String>> problems = new TreeMap<String,SortedSet<String>>(); 
+    NodeList nodeList = getNodeList(elementXpath, false);
+    for (int i = 0 ; i < nodeList.getLength() ; ++i) {
+      Element element = (Element)nodeList.item(i);
+      Set<String> unknownAttributes = getUnknownAttributes(element, knownAttributes);
+      if (null != unknownAttributes) {
+        String elementName = element.getNodeName();
+        SortedSet<String> allUnknownAttributes = problems.get(elementName);
+        if (null == allUnknownAttributes) {
+          allUnknownAttributes = new TreeSet<String>();
+          problems.put(elementName, allUnknownAttributes);
+        }
+        allUnknownAttributes.addAll(unknownAttributes);
+      }
+    }
+    if (problems.size() > 0) {
+      StringBuilder message = new StringBuilder();
+      for (Map.Entry<String,SortedSet<String>> entry : problems.entrySet()) {
+        if (message.length() > 0) {
+          message.append(", ");
+        }
+        message.append('<');
+        message.append(entry.getKey());
+        for (String attributeName : entry.getValue()) {
+          message.append(' ');
+          message.append(attributeName);
+          message.append("=\"...\"");
+        }
+        message.append('>');
+      }
+      message.insert(0, "Unknown attribute(s) on element(s): ");
+      String msg = message.toString();
+      SolrException.log(log, msg);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
+    }
+  }
+
   public String getVal(String path, boolean errIfMissing) {
     Node nd = getNode(path,errIfMissing);
     if (nd==null) return null;

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/core/ConfigSolr.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/core/ConfigSolr.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/core/ConfigSolr.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/core/ConfigSolr.java Thu May 30 07:53:18 2013
@@ -17,67 +17,127 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import org.apache.solr.cloud.ZkController;
-import org.apache.solr.handler.component.ShardHandlerFactory;
-import org.apache.solr.schema.IndexSchema;
-import org.apache.zookeeper.KeeperException;
-
-import java.io.File;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-/**
- * ConfigSolr is a new interface  to aid us in obsoleting solr.xml and replacing it with solr.properties. The problem here
- * is that the Config class is used for _all_ the xml file, e.g. solrconfig.xml and we can't mess with _that_ as part
- * of this issue. Primarily used in CoreContainer at present.
- * <p/>
- * This is already deprecated, it's only intended to exist for while transitioning to properties-based replacement for
- * solr.xml
- *
- * @since solr 4.2
- */
-@Deprecated
-public interface ConfigSolr {
-
-  public static enum ConfLevel {
-    SOLR, SOLR_CORES, SOLR_CORES_CORE, SOLR_LOGGING, SOLR_LOGGING_WATCHER
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.util.DOMUtil;
+import org.apache.solr.util.PropertiesUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+
+public abstract class ConfigSolr {
+  protected static Logger log = LoggerFactory.getLogger(ConfigSolr.class);
+  
+  public final static String SOLR_XML_FILE = "solr.xml";
+  
+  // Ugly for now, but we'll at least be able to centralize all of the differences between 4x and 5x.
+  public static enum CfgProp {
+    SOLR_ADMINHANDLER,
+    SOLR_CORELOADTHREADS,
+    SOLR_COREROOTDIRECTORY,
+    SOLR_DISTRIBUPDATECONNTIMEOUT,
+    SOLR_DISTRIBUPDATESOTIMEOUT,
+    SOLR_HOST,
+    SOLR_HOSTCONTEXT,
+    SOLR_HOSTPORT,
+    SOLR_LEADERVOTEWAIT,
+    SOLR_LOGGING_CLASS,
+    SOLR_LOGGING_ENABLED,
+    SOLR_LOGGING_WATCHER_SIZE,
+    SOLR_LOGGING_WATCHER_THRESHOLD,
+    SOLR_MANAGEMENTPATH,
+    SOLR_SHAREDLIB,
+    SOLR_SHARDHANDLERFACTORY_CLASS,
+    SOLR_SHARDHANDLERFACTORY_CONNTIMEOUT,
+    SOLR_SHARDHANDLERFACTORY_NAME,
+    SOLR_SHARDHANDLERFACTORY_SOCKETTIMEOUT,
+    SOLR_SHARESCHEMA,
+    SOLR_TRANSIENTCACHESIZE,
+    SOLR_ZKCLIENTTIMEOUT,
+    SOLR_ZKHOST,
+
+    //TODO: Remove all of these elements for 5.0
+    SOLR_PERSISTENT,
+    SOLR_CORES_DEFAULT_CORE_NAME,
+    SOLR_ADMINPATH
   }
 
-  ;
-
-  public int getInt(ConfLevel level, String tag, int def);
+  protected Config config;
+  protected Map<CfgProp, String> propMap = new HashMap<CfgProp, String>();
 
-  public boolean getBool(ConfLevel level, String tag, boolean defValue);
-
-  public String get(ConfLevel level, String tag, String def);
-
-  public void substituteProperties();
+  public ConfigSolr(Config config) {
+    this.config = config;
+  }
+  
+  public Config getConfig() {
+    return config;
+  }
 
-  public ShardHandlerFactory initShardHandler();
+  public int getInt(CfgProp prop, int def) {
+    String val = propMap.get(prop);
+    if (val != null) val = PropertiesUtil.substituteProperty(val, null);
+    return (val == null) ? def : Integer.parseInt(val);
+  }
 
-  public Properties getSolrProperties(ConfigSolr cfg, String context);
+  public boolean getBool(CfgProp prop, boolean defValue) {
+    String val = propMap.get(prop);
+    if (val != null) val = PropertiesUtil.substituteProperty(val, null);
+    return (val == null) ? defValue : Boolean.parseBoolean(val);
+  }
 
-  public IndexSchema getSchemaFromZk(ZkController zkController, String zkConfigName, String schemaName,
-                                     SolrConfig config) throws KeeperException, InterruptedException;
+  public String get(CfgProp prop, String def) {
+    String val = propMap.get(prop);
+    if (val != null) val = PropertiesUtil.substituteProperty(val, null);
+    return (val == null) ? def : val;
+  }
 
-  public SolrConfig getSolrConfigFromZk(ZkController zkController, String zkConfigName, String solrConfigFileName,
-                                        SolrResourceLoader resourceLoader);
+  // For saving the original property, ${} syntax and all.
+  public String getOrigProp(CfgProp prop, String def) {
+    String val = propMap.get(prop);
+    return (val == null) ? def : val;
+  }
 
-  public void initPersist();
+  public Properties getSolrProperties(String path) {
+    try {
+      return readProperties(((NodeList) config.evaluate(
+          path, XPathConstants.NODESET)).item(0));
+    } catch (Throwable e) {
+      SolrException.log(log, null, e);
+    }
+    return null;
 
-  public void addPersistCore(String coreName, Properties attribs, Map<String, String> props);
+  }
+  
+  protected Properties readProperties(Node node) throws XPathExpressionException {
+    XPath xpath = config.getXPath();
+    NodeList props = (NodeList) xpath.evaluate("property", node, XPathConstants.NODESET);
+    Properties properties = new Properties();
+    for (int i = 0; i < props.getLength(); i++) {
+      Node prop = props.item(i);
+      properties.setProperty(DOMUtil.getAttr(prop, "name"), DOMUtil.getAttr(prop, "value"));
+    }
+    return properties;
+  }
 
-  public void addPersistAllCores(Properties containerProperties, Map<String, String> rootSolrAttribs, Map<String, String> coresAttribs,
-                                 File file);
+  public abstract void substituteProperties();
 
-  public String getCoreNameFromOrig(String origCoreName, SolrResourceLoader loader, String coreName);
+  public abstract List<String> getAllCoreNames();
 
-  public List<String> getAllCoreNames();
+  public abstract String getProperty(String coreName, String property, String defaultVal);
 
-  public String getProperty(String coreName, String property, String defaultVal);
+  public abstract Properties readCoreProperties(String coreName);
 
-  public Properties readCoreProperties(String coreName);
+  public abstract Map<String, String> readCoreAttributes(String coreName);
 
-  public Map<String, String> readCoreAttributes(String coreName);
 }
+