You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2011/02/27 13:52:06 UTC

svn commit: r1075044 - in /lucene/dev/branches/branch_3x/solr: ./ contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ src/common/org/apache/solr/common/util/ src/java/org/apache/solr/core/ src/java/org/apache/solr/handler/compon...

Author: uschindler
Date: Sun Feb 27 12:52:05 2011
New Revision: 1075044

URL: http://svn.apache.org/viewvc?rev=1075044&view=rev
Log:
SOLR-1656: XIncludes and other HREFs in XML files loaded by ResourceLoader are fixed to be resolved using the URI standard (RFC 2396)

Added:
    lucene/dev/branches/branch_3x/solr/src/common/org/apache/solr/common/util/SystemIdResolver.java   (with props)
    lucene/dev/branches/branch_3x/solr/src/test-files/solr/conf/xslt/dummy-using-include.xsl   (with props)
    lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/common/util/TestSystemIdResolver.java   (with props)
Modified:
    lucene/dev/branches/branch_3x/solr/CHANGES.txt
    lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandler.java
    lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java
    lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java
    lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/Config.java
    lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/SolrConfig.java
    lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
    lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/IndexSchema.java
    lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/util/xslt/TransformerProvider.java
    lucene/dev/branches/branch_3x/solr/src/test-files/solr/conf/solrconfig-xinclude.xml
    lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/OutputWriterTest.java
    lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/core/TestXIncludeConfig.java
    lucene/dev/branches/branch_3x/solr/src/webapp/src/org/apache/solr/servlet/SolrDispatchFilter.java

Modified: lucene/dev/branches/branch_3x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/CHANGES.txt?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/solr/CHANGES.txt Sun Feb 27 12:52:05 2011
@@ -519,7 +519,7 @@ Bug Fixes
 
 * SOLR-96: Fix XML parsing in XMLUpdateRequestHandler and
   DocumentAnalysisRequestHandler to respect charset from XML file and only
-  use HTTP header's "Content-Type" as a "hint". (Uwe Schindler)
+  use HTTP header's "Content-Type" as a "hint". (uschindler)
 
 * SOLR-2339: Fix sorting to explicitly generate an error if you
   attempt to sort on a multiValued field. (hossman)
@@ -535,6 +535,19 @@ Bug Fixes
 * SOLR-2380: Distributed faceting could miss values when facet.sort=index
   and when facet.offset was greater than 0. (yonik)
   
+* SOLR-1656: XIncludes and other HREFs in XML files loaded by ResourceLoader
+  are fixed to be resolved using the URI standard (RFC 2396). The system 
+  identifier is no longer a plain filename with path, it gets initialized 
+  using a custom URI scheme "solrres:". This scheme is resolved using a 
+  EntityResolver that utilizes ResourceLoader 
+  (org.apache.solr.common.util.SystemIdResolver). This makes all relative 
+  pathes in Solr's config files behave like expected. This change 
+  introduces some backwards breaks in the API: Some config classes 
+  (Config, SolrConfig, IndexSchema) were changed to take 
+  org.xml.sax.InputSource instead of InputStream. There may also be some 
+  backwards breaks in existing config files, it is recommended to check 
+  your config files / XSLTs and replace all XIncludes/HREFs that were 
+  hacked to use absolute paths to use relative ones. (uschindler)
 
 
 Other Changes

Modified: lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandler.java?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandler.java (original)
+++ lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandler.java Sun Feb 27 12:52:05 2011
@@ -27,6 +27,7 @@ import org.apache.solr.common.util.Conte
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.SystemIdResolver;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
@@ -41,8 +42,10 @@ import org.apache.solr.update.processor.
 import org.apache.solr.util.plugin.SolrCoreAware;
 
 import java.util.*;
+import java.io.StringReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.xml.sax.InputSource;
 
 /**
  * <p>
@@ -108,9 +111,9 @@ public class DataImportHandler extends R
         String configLoc = (String) defaults.get("config");
         if (configLoc != null && configLoc.length() != 0) {
           processConfiguration(defaults);
-
-          importer = new DataImporter(SolrWriter.getResourceAsString(core
-                  .getResourceLoader().openResource(configLoc)), core,
+          final InputSource is = new InputSource(core.getResourceLoader().openConfig(configLoc));
+          is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(configLoc));
+          importer = new DataImporter(is, core,
                   dataSources, coreScopeSession);
         }
       }
@@ -162,7 +165,7 @@ public class DataImportHandler extends R
       if (requestParams.dataConfig != null) {
         try {
           processConfiguration((NamedList) initArgs.get("defaults"));
-          importer = new DataImporter(requestParams.dataConfig, req.getCore()
+          importer = new DataImporter(new InputSource(new StringReader(requestParams.dataConfig)), req.getCore()
                   , dataSources, coreScopeSession);
         } catch (RuntimeException e) {
           rsp.add("exception", DebugLogger.getStacktraceString(e));

Modified: lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java (original)
+++ lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java Sun Feb 27 12:52:05 2011
@@ -24,6 +24,7 @@ import org.apache.solr.schema.IndexSchem
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.SystemIdResolver;
 
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
@@ -33,6 +34,7 @@ import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
+import org.apache.commons.io.IOUtils;
 
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -88,7 +90,7 @@ public class DataImporter {
     coreScopeSession = new ConcurrentHashMap<String, Object>();
   }
 
-  DataImporter(String dataConfig, SolrCore core, Map<String, Properties> ds, Map<String, Object> session) {
+  DataImporter(InputSource dataConfig, SolrCore core, Map<String, Properties> ds, Map<String, Object> session) {
     if (dataConfig == null)
       throw new DataImportHandlerException(SEVERE,
               "Configuration not found");
@@ -141,7 +143,7 @@ public class DataImporter {
    * Used by tests
    */
   void loadAndInit(String configStr) {
-    loadDataConfig(configStr);
+    loadDataConfig(new InputSource(new StringReader(configStr)));
     Map<String, DataConfig.Field> fields = new HashMap<String, DataConfig.Field>();
     for (DataConfig.Entity entity : config.document.entities) {
       initEntity(entity, fields, false);
@@ -168,19 +170,31 @@ public class DataImporter {
 
   }
 
-  private void loadDataConfig(String configFile) {
+  private void loadDataConfig(InputSource configFile) {
 
     try {
       DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
-      try {
-        dbf.setXIncludeAware(true);
-        dbf.setNamespaceAware(true);
-      } catch( UnsupportedOperationException e ) {
-        LOG.warn( "XML parser doesn't support XInclude option" );
+      
+      // only enable xinclude, if a a SolrCore and SystemId is present (makes no sense otherwise)
+      if (core != null && configFile.getSystemId() != null) {
+        try {
+          dbf.setXIncludeAware(true);
+          dbf.setNamespaceAware(true);
+        } catch( UnsupportedOperationException e ) {
+          LOG.warn( "XML parser doesn't support XInclude option" );
+        }
       }
+      
       DocumentBuilder builder = dbf.newDocumentBuilder();
-      Document document = builder.parse(new InputSource(new StringReader(
-              configFile)));
+      if (core != null)
+        builder.setEntityResolver(new SystemIdResolver(core.getResourceLoader()));
+      Document document;
+      try {
+        document = builder.parse(configFile);
+      } finally {
+        // some XML parsers are broken and don't close the byte stream (but they should according to spec)
+        IOUtils.closeQuietly(configFile.getByteStream());
+      }
 
       config = new DataConfig();
       NodeList elems = document.getElementsByTagName("dataConfig");

Modified: lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java (original)
+++ lucene/dev/branches/branch_3x/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java Sun Feb 27 12:52:05 2011
@@ -18,8 +18,12 @@ package org.apache.solr.handler.dataimpo
 
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.common.ResourceLoader;
+import org.apache.solr.common.util.SystemIdResolver;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.commons.io.IOUtils;
 
 import javax.xml.transform.Source;
 import javax.xml.transform.TransformerException;
@@ -107,12 +111,26 @@ public class XPathEntityProcessor extend
     if (xslt != null) {
       xslt = context.replaceTokens(xslt);
       try {
-        Source xsltSource = new StreamSource(xslt);
         // create an instance of TransformerFactory
         TransformerFactory transFact = TransformerFactory.newInstance();
-        xslTransformer = transFact.newTransformer(xsltSource);
-        LOG
-                .info("Using xslTransformer: "
+        final SolrCore core = context.getSolrCore();
+        final StreamSource xsltSource;
+        if (core != null) {
+          final ResourceLoader loader = core.getResourceLoader();
+          transFact.setURIResolver(new SystemIdResolver(loader).asURIResolver());
+          xsltSource = new StreamSource(loader.openResource(xslt),
+            SystemIdResolver.createSystemIdFromResourceName(xslt));
+        } else {
+          // fallback for tests
+          xsltSource = new StreamSource(xslt);
+        }
+        try {
+          xslTransformer = transFact.newTransformer(xsltSource);
+        } finally {
+          // some XML parsers are broken and don't close the byte stream (but they should according to spec)
+          IOUtils.closeQuietly(xsltSource.getInputStream());
+        }
+        LOG.info("Using xslTransformer: "
                         + xslTransformer.getClass().getName());
       } catch (Exception e) {
         throw new DataImportHandlerException(SEVERE,

Added: lucene/dev/branches/branch_3x/solr/src/common/org/apache/solr/common/util/SystemIdResolver.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/common/org/apache/solr/common/util/SystemIdResolver.java?rev=1075044&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/common/org/apache/solr/common/util/SystemIdResolver.java (added)
+++ lucene/dev/branches/branch_3x/solr/src/common/org/apache/solr/common/util/SystemIdResolver.java Sun Feb 27 12:52:05 2011
@@ -0,0 +1,171 @@
+/**
+ * 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.util;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.solr.common.ResourceLoader;
+
+import org.xml.sax.InputSource;
+import org.xml.sax.EntityResolver;
+import org.xml.sax.ext.EntityResolver2;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import javax.xml.transform.Source;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.URIResolver;
+import javax.xml.transform.sax.SAXSource;
+import javax.xml.stream.XMLResolver;
+import javax.xml.stream.XMLStreamException;
+
+/**
+ * This is a helper class to support resolving of XIncludes or other hrefs
+ * inside XML files on top of a {@link ResourceLoader}. Just plug this class
+ * on top of a {@link ResourceLoader} and pass it as {@link EntityResolver} to SAX parsers
+ * or via wrapper methods as {@link URIResolver} to XSL transformers or {@link XMLResolver} to STAX parsers.
+ * The resolver handles special SystemIds with an URI scheme of {@code solrres:} that point
+ * to resources. To produce such systemIds when you initially call the parser, use
+ * {@link #createSystemIdFromResourceName} which produces a SystemId that can
+ * be included along the InputStream coming from {@link ResourceLoader#openResource}.
+ * <p>In general create the {@link InputSource} to be passed to the parser like:</p>
+ * <pre class="prettyprint">
+ *  InputSource is = new InputSource(loader.openSchema(name));
+ *  is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
+ *  final DocumentBuilder db = DocumentBuilderFactory.newInstance().newDocumentBuilder();
+ *  db.setEntityResolver(new SystemIdResolver(loader));
+ *  Document doc = db.parse(is);
+ * </pre>
+ */
+public final class SystemIdResolver implements EntityResolver, EntityResolver2 {
+  private static final Logger log = LoggerFactory.getLogger(SystemIdResolver.class);
+
+  public static final String RESOURCE_LOADER_URI_SCHEME = "solrres";
+  public static final String RESOURCE_LOADER_AUTHORITY_ABSOLUTE = "@";
+
+  private final ResourceLoader loader;
+
+  public SystemIdResolver(ResourceLoader loader) {
+    this.loader = loader;
+  }
+  
+  public EntityResolver asEntityResolver() {
+    return this;
+  }
+  
+  public URIResolver asURIResolver() {
+    return new URIResolver() {
+      public Source resolve(String href, String base) throws TransformerException {
+        try {
+          final InputSource src = SystemIdResolver.this.resolveEntity(null, null, base, href);
+          return (src == null) ? null : new SAXSource(src);
+        } catch (IOException ioe) {
+          throw new TransformerException("Cannot resolve entity", ioe);
+        }
+      }
+    };
+  }
+  
+  public XMLResolver asXMLResolver() {
+    return new XMLResolver() {
+      public Object resolveEntity(String publicId, String systemId, String baseURI, String namespace) throws XMLStreamException {
+        try {
+          final InputSource src = SystemIdResolver.this.resolveEntity(null, publicId, baseURI, systemId);
+          return (src == null) ? null : src.getByteStream();
+        } catch (IOException ioe) {
+          throw new XMLStreamException("Cannot resolve entity", ioe);
+        }
+      }
+    };
+  }
+  
+  URI resolveRelativeURI(String baseURI, String systemId) throws IOException,URISyntaxException {
+    URI uri;
+    
+    // special case for backwards compatibility: if relative systemId starts with "/" (we convert that to an absolute solrres:-URI)
+    if (systemId.startsWith("/")) {
+      uri = new URI(RESOURCE_LOADER_URI_SCHEME, RESOURCE_LOADER_AUTHORITY_ABSOLUTE, "/", null, null).resolve(systemId);
+    } else {
+      // simply parse as URI
+      uri = new URI(systemId);
+    }
+    
+    // do relative resolving
+    if (baseURI != null ) {
+      uri = new URI(baseURI).resolve(uri);
+    }
+    
+    return uri;
+  }
+  
+  // *** EntityResolver(2) methods:
+  
+  public InputSource getExternalSubset(String name, String baseURI) {
+    return null;
+  }
+  
+  public InputSource resolveEntity(String name, String publicId, String baseURI, String systemId) throws IOException {
+    if (systemId == null)
+      return null;
+    try {
+      final URI uri = resolveRelativeURI(baseURI, systemId);
+      
+      // check schema and resolve with ResourceLoader
+      if (RESOURCE_LOADER_URI_SCHEME.equals(uri.getScheme())) {
+        String path = uri.getPath(), authority = uri.getAuthority();
+        if (!RESOURCE_LOADER_AUTHORITY_ABSOLUTE.equals(authority)) {
+          path = path.substring(1);
+        }
+        final InputSource is = new InputSource(loader.openResource(path));
+        is.setSystemId(uri.toASCIIString());
+        is.setPublicId(publicId);
+        return is;
+      } else {
+        // resolve all other URIs using the standard resolver
+        return null;
+      }
+    } catch (URISyntaxException use) {
+      log.warn("An URI systax problem occurred during resolving SystemId, falling back to default resolver", use);
+      return null;
+    }
+  }
+
+  public InputSource resolveEntity(String publicId, String systemId) throws IOException {
+    return resolveEntity(null, publicId, null, systemId);
+  }
+  
+  public static String createSystemIdFromResourceName(String name) {
+    name = name.replace(File.separatorChar, '/');
+    final String authority;
+    if (name.startsWith("/")) {
+      // a hack to preserve absolute filenames and keep them absolute after resolving, we set the URI's authority to "@" on absolute filenames:
+      authority = RESOURCE_LOADER_AUTHORITY_ABSOLUTE;
+    } else {
+      authority = null;
+      name = "/" + name;
+    }
+    try {
+      return new URI(RESOURCE_LOADER_URI_SCHEME, authority, name, null, null).toASCIIString();
+    } catch (URISyntaxException use) {
+      throw new IllegalArgumentException("Invalid syntax of Solr Resource URI", use);
+    }
+  }
+
+}

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/Config.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/Config.java?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/Config.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/Config.java Sun Feb 27 12:52:05 2011
@@ -17,26 +17,32 @@
 
 package org.apache.solr.core;
 
+import org.apache.lucene.util.Version;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.DOMUtil;
+import org.apache.solr.common.util.SystemIdResolver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.w3c.dom.Document;
 import org.w3c.dom.Node;
+import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.DOMUtil;
+import org.apache.commons.io.IOUtils;
 
-import javax.xml.parsers.*;
+import javax.xml.namespace.QName;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathFactory;
 import javax.xml.xpath.XPathConstants;
 import javax.xml.xpath.XPathExpressionException;
-import javax.xml.namespace.QName;
-import java.io.*;
+import javax.xml.xpath.XPathFactory;
+import java.io.IOException;
+import java.io.InputStream;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Locale;
 import java.util.concurrent.atomic.AtomicBoolean;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.lucene.util.Version;
 
 /**
  * @version $Id$
@@ -52,12 +58,12 @@ public class Config {
   private final SolrResourceLoader loader;
 
   /**
-   * @deprecated Use {@link #Config(SolrResourceLoader, String, InputStream, String)} instead. 
+   * @deprecated Use {@link #Config(SolrResourceLoader, String, InputSource, String)} instead. 
    */
   @Deprecated
   public Config(String name, InputStream is, String prefix) throws ParserConfigurationException, IOException, SAXException 
   {
-    this( null, name, is, prefix );
+    this( null, name, new InputSource(is), prefix );
   }
 
   /**
@@ -70,7 +76,7 @@ public class Config {
    */
   public Config(SolrResourceLoader loader, String name) throws ParserConfigurationException, IOException, SAXException 
   {
-    this( loader, name, null, null );
+    this( loader, name, (InputSource) null, null );
   }
   
   /**
@@ -91,30 +97,69 @@ public class Config {
    * @throws javax.xml.parsers.ParserConfigurationException
    * @throws java.io.IOException
    * @throws org.xml.sax.SAXException
+   * @deprecated Use {@link #Config(SolrResourceLoader, String, InputSource, String)} instead. 
    */
+  @Deprecated
   public Config(SolrResourceLoader loader, String name, InputStream is, String prefix) throws ParserConfigurationException, IOException, SAXException 
   {
+   this(loader, name, (is == null) ? null : new InputSource(is), prefix);
+  }
+  
+  /**
+   * Builds a config:
+   * <p>
+   * Note that the 'name' parameter is used to obtain a valid input stream if no valid one is provided through 'is'.
+   * If no valid stream is provided, a valid SolrResourceLoader instance should be provided through 'loader' so
+   * the resource can be opened (@see SolrResourceLoader#openResource); if no SolrResourceLoader instance is provided, a default one
+   * will be created.
+   * </p>
+   * <p>
+   * Consider passing a non-null 'name' parameter in all use-cases since it is used for logging & exception reporting.
+   * </p>
+   * @param loader the resource loader used to obtain an input stream if 'is' is null
+   * @param name the resource name used if the input stream 'is' is null
+   * @param is the resource as a SAX InputSource
+   * @param prefix an optional prefix that will be preprended to all non-absolute xpath expressions
+   * @throws javax.xml.parsers.ParserConfigurationException
+   * @throws java.io.IOException
+   * @throws org.xml.sax.SAXException
+   */
+  public Config(SolrResourceLoader loader, String name, InputSource is, String prefix) throws ParserConfigurationException, IOException, SAXException 
+  {
     if( loader == null ) {
       loader = new SolrResourceLoader( null );
     }
     this.loader = loader;
     this.name = name;
     this.prefix = (prefix != null && !prefix.endsWith("/"))? prefix + '/' : prefix;
-    InputStream lis = is;
     try {
-      if (lis == null) {
-        lis = loader.openConfig(name);
-      }
       javax.xml.parsers.DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+      
+      if (is == null) {
+        is = new InputSource(loader.openConfig(name));
+        is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
+      }
+
+      // only enable xinclude, if a SystemId is available
+      if (is.getSystemId() != null) {
+        try {
+          dbf.setXIncludeAware(true);
+          dbf.setNamespaceAware(true);
+        } catch(UnsupportedOperationException e) {
+          log.warn(name + " XML parser doesn't support XInclude option");
+        }
+      }
+      
+      final DocumentBuilder db = dbf.newDocumentBuilder();
+      db.setEntityResolver(new SystemIdResolver(loader));
       try {
-        dbf.setXIncludeAware(true);
-        dbf.setNamespaceAware(true);
-      } catch(UnsupportedOperationException e) {
-        log.warn(name + " XML parser doesn't support XInclude option");
+        doc = db.parse(is);
+      } finally {
+        // some XML parsers are broken and don't close the byte stream (but they should according to spec)
+        IOUtils.closeQuietly(is.getByteStream());
       }
-      doc = dbf.newDocumentBuilder().parse(lis);
 
-        DOMUtil.substituteProperties(doc, loader.getCoreProperties());
+      DOMUtil.substituteProperties(doc, loader.getCoreProperties());
     } catch (ParserConfigurationException e)  {
       SolrException.log(log, "Exception during parsing file: " + name, e);
       throw e;
@@ -124,9 +169,6 @@ public class Config {
     } catch( SolrException e ){
     	SolrException.log(log,"Error in "+name,e);
     	throw e;
-    } finally {
-      // if this opens the resource, it also closes it
-      if (lis != is)  lis.close();
     }
   }
 

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/CoreContainer.java?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/CoreContainer.java Sun Feb 27 12:52:05 2011
@@ -42,6 +42,7 @@ import org.apache.commons.io.IOUtils;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
+import org.xml.sax.InputSource;
 
 
 /**
@@ -129,7 +130,7 @@ public class CoreContainer 
         cores.load(solrHome, fconf);
       } else {
         cores.defaultAbortOnConfigError = abortOnConfigurationError;
-        cores.load(solrHome, new ByteArrayInputStream(DEF_SOLR_XML.getBytes("UTF-8")));
+        cores.load(solrHome, new InputSource(new ByteArrayInputStream(DEF_SOLR_XML.getBytes("UTF-8"))));
         cores.configFile = fconf;
       }
       setAbortOnConfigurationError(0 < cores.numCoresAbortOnConfigError);
@@ -203,7 +204,7 @@ public class CoreContainer 
    */
   public void load(String dir, File configFile ) throws ParserConfigurationException, IOException, SAXException {
     this.configFile = configFile;
-    this.load(dir, new FileInputStream(configFile));
+    this.load(dir, new InputSource(configFile.toURI().toASCIIString()));
   } 
 
   /**
@@ -215,117 +216,109 @@ public class CoreContainer 
    * @throws IOException
    * @throws SAXException
    */
-  public void load(String dir, InputStream cfgis)
+  public void load(String dir, InputSource cfgis)
       throws ParserConfigurationException, IOException, SAXException {
     this.loader = new SolrResourceLoader(dir);
     solrHome = loader.getInstanceDir();
+    Config cfg = new Config(loader, null, cfgis, null);
+    String dcoreName = cfg.get("solr/cores/@defaultCoreName", null);
+    if(dcoreName != null) {
+      defaultCoreName = dcoreName;
+    }
+    persistent = cfg.getBool( "solr/@persistent", false );
+    libDir     = cfg.get(     "solr/@sharedLib", null);
+    adminPath  = cfg.get(     "solr/cores/@adminPath", null );
+    shareSchema = cfg.getBool("solr/cores/@shareSchema", false );
+    if(shareSchema){
+      indexSchemaCache = new ConcurrentHashMap<String ,IndexSchema>();
+    }
+    adminHandler  = cfg.get("solr/cores/@adminHandler", null );
+    managementPath  = cfg.get("solr/cores/@managementPath", null );
+
+    if (libDir != null) {
+      File f = FileUtils.resolvePath(new File(dir), libDir);
+      log.info( "loading shared library: "+f.getAbsolutePath() );
+      libLoader = SolrResourceLoader.createClassLoader(f, null);
+    }
+
+    if (adminPath != null) {
+      if (adminHandler == null) {
+        coreAdminHandler = new CoreAdminHandler(this);
+      } else {
+        coreAdminHandler = this.createMultiCoreHandler(adminHandler);
+      }
+    }
+
     try {
-      Config cfg = new Config(loader, null, cfgis, null);
-      String dcoreName = cfg.get("solr/cores/@defaultCoreName", null);
-      if(dcoreName != null) {
-        defaultCoreName = dcoreName;
-      }
-      persistent = cfg.getBool( "solr/@persistent", false );
-      libDir     = cfg.get(     "solr/@sharedLib", null);
-      adminPath  = cfg.get(     "solr/cores/@adminPath", null );
-      shareSchema = cfg.getBool("solr/cores/@shareSchema", false );
-      if(shareSchema){
-        indexSchemaCache = new ConcurrentHashMap<String ,IndexSchema>();
-      }
-      adminHandler  = cfg.get("solr/cores/@adminHandler", null );
-      managementPath  = cfg.get("solr/cores/@managementPath", null );
-
-      if (libDir != null) {
-        File f = FileUtils.resolvePath(new File(dir), libDir);
-        log.info( "loading shared library: "+f.getAbsolutePath() );
-        libLoader = SolrResourceLoader.createClassLoader(f, null);
-      }
-
-      if (adminPath != null) {
-        if (adminHandler == null) {
-          coreAdminHandler = new CoreAdminHandler(this);
-        } else {
-          coreAdminHandler = this.createMultiCoreHandler(adminHandler);
+      containerProperties = readProperties(cfg, ((NodeList) cfg.evaluate("solr", XPathConstants.NODESET)).item(0));
+    } catch (Throwable e) {
+      SolrConfig.severeErrors.add(e);
+      SolrException.logOnce(log,null,e);
+    }
+    
+    // before looping over each core, let's check the names and fail 
+    // fast if the same one is reused multiple times.
+    { // local scope, won't need these vars again
+      NodeList nodes = (NodeList)cfg.evaluate("solr/cores/core/@name", 
+                                              XPathConstants.NODESET);
+      Set<String> names = new HashSet<String>();
+      for (int i=0; i<nodes.getLength(); i++) {
+        String name = DOMUtil.getText(nodes.item(i));
+        if (names.contains(name)) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+                                  "Multiple cores found with same name: " + 
+                                  name);
         }
+        names.add(name);
       }
+    }
 
+    NodeList nodes = (NodeList)cfg.evaluate("solr/cores/core", XPathConstants.NODESET);
+    for (int i=0; i<nodes.getLength(); i++) {
+      Node node = nodes.item(i);
       try {
-        containerProperties = readProperties(cfg, ((NodeList) cfg.evaluate("solr", XPathConstants.NODESET)).item(0));
-      } catch (Throwable e) {
-        SolrConfig.severeErrors.add(e);
-        SolrException.logOnce(log,null,e);
-      }
-      
-      // before looping over each core, let's check the names and fail 
-      // fast if the same one is reused multiple times.
-      { // local scope, won't need these vars again
-        NodeList nodes = (NodeList)cfg.evaluate("solr/cores/core/@name", 
-                                                XPathConstants.NODESET);
-        Set<String> names = new HashSet<String>();
-        for (int i=0; i<nodes.getLength(); i++) {
-          String name = DOMUtil.getText(nodes.item(i));
-          if (names.contains(name)) {
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-                                    "Multiple cores found with same name: " + 
-                                    name);
-          }
-          names.add(name);
+        String name = DOMUtil.getAttr(node, "name", null);
+        if (null == name) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+                                  "Each core in solr.xml must have a 'name'");
+        }
+        if (name.equals(defaultCoreName)){
+          // for the default core we use a blank name,
+          // later on attempts to access it by it's full name will 
+          // be mapped to this.
+          name="";
         }
-      }
 
-      NodeList nodes = (NodeList)cfg.evaluate("solr/cores/core", XPathConstants.NODESET);
-      for (int i=0; i<nodes.getLength(); i++) {
-        Node node = nodes.item(i);
-        try {
-          String name = DOMUtil.getAttr(node, "name", null);
-          if (null == name) {
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-                                    "Each core in solr.xml must have a 'name'");
-          }
-          if (name.equals(defaultCoreName)){
-            // for the default core we use a blank name,
-            // later on attempts to access it by it's full name will 
-            // be mapped to this.
-            name="";
-          }
-
-          CoreDescriptor p = new CoreDescriptor(this, name, DOMUtil.getAttr(node, "instanceDir", null));
-
-          // deal with optional settings
-          String opt = DOMUtil.getAttr(node, "config", null);
-          if(solrConfigFilenameOverride != null && name.equals("")) {
-            p.setConfigName(solrConfigFilenameOverride);
-          } else if (opt != null) {
-            p.setConfigName(opt);
-          }
-          opt = DOMUtil.getAttr(node, "schema", null);
-          if (opt != null) {
-            p.setSchemaName(opt);
-          }
-          opt = DOMUtil.getAttr(node, "properties", null);
-          if (opt != null) {
-            p.setPropertiesName(opt);
-          }
-          opt = DOMUtil.getAttr(node, CoreAdminParams.DATA_DIR, null);
-          if (opt != null) {
-            p.setDataDir(opt);
-          }
-
-          p.setCoreProperties(readProperties(cfg, node));
-
-          SolrCore core = create(p);
-          register(name, core, false);
-        }
-        catch (Throwable ex) {
-          SolrConfig.severeErrors.add( ex );
-          SolrException.logOnce(log,null,ex);
+        CoreDescriptor p = new CoreDescriptor(this, name, DOMUtil.getAttr(node, "instanceDir", null));
+
+        // deal with optional settings
+        String opt = DOMUtil.getAttr(node, "config", null);
+        if(solrConfigFilenameOverride != null && name.equals("")) {
+          p.setConfigName(solrConfigFilenameOverride);
+        } else if (opt != null) {
+          p.setConfigName(opt);
+        }
+        opt = DOMUtil.getAttr(node, "schema", null);
+        if (opt != null) {
+          p.setSchemaName(opt);
+        }
+        opt = DOMUtil.getAttr(node, "properties", null);
+        if (opt != null) {
+          p.setPropertiesName(opt);
+        }
+        opt = DOMUtil.getAttr(node, CoreAdminParams.DATA_DIR, null);
+        if (opt != null) {
+          p.setDataDir(opt);
         }
-      }
-    }
 
-    finally {
-      if (cfgis != null) {
-        try { cfgis.close(); } catch (Exception xany) {}
+        p.setCoreProperties(readProperties(cfg, node));
+
+        SolrCore core = create(p);
+        register(name, core, false);
+      }
+      catch (Throwable ex) {
+        SolrConfig.severeErrors.add( ex );
+        SolrException.logOnce(log,null,ex);
       }
     }
   }

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/SolrConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/SolrConfig.java?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/SolrConfig.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/core/SolrConfig.java Sun Feb 27 12:52:05 2011
@@ -44,6 +44,7 @@ import org.slf4j.LoggerFactory;
 
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
+import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
 
 import javax.xml.parsers.ParserConfigurationException;
@@ -104,7 +105,7 @@ public class SolrConfig extends Config {
    *@param name the configuration name
    *@param is the configuration stream
    */
-  public SolrConfig(String name, InputStream is)
+  public SolrConfig(String name, InputSource is)
   throws ParserConfigurationException, IOException, SAXException {
     this( (SolrResourceLoader) null, name, is );
   }
@@ -114,7 +115,7 @@ public class SolrConfig extends Config {
    *@param name the configuration name used by the loader if the stream is null
    *@param is the configuration stream 
    */
-  public SolrConfig(String instanceDir, String name, InputStream is)
+  public SolrConfig(String instanceDir, String name, InputSource is)
   throws ParserConfigurationException, IOException, SAXException {
     this(new SolrResourceLoader(instanceDir), name, is);
   }
@@ -126,7 +127,7 @@ public class SolrConfig extends Config {
    *@param name the configuration name
    *@param is the configuration stream
    */
-  SolrConfig(SolrResourceLoader loader, String name, InputStream is)
+  SolrConfig(SolrResourceLoader loader, String name, InputSource is)
   throws ParserConfigurationException, IOException, SAXException {
     super(loader, name, is, "/config/");
     initLibs();

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/handler/component/QueryElevationComponent.java?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/handler/component/QueryElevationComponent.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/handler/component/QueryElevationComponent.java Sun Feb 27 12:52:05 2011
@@ -64,6 +64,7 @@ import org.apache.solr.util.plugin.SolrC
 import org.apache.solr.request.SolrQueryRequest;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
+import org.xml.sax.InputSource;
 
 /**
  * A component to elevate some documents to the top of the result set.
@@ -216,7 +217,7 @@ public class QueryElevationComponent ext
         log.info( "Loading QueryElevation from data dir: "+f );
 
         InputStream is = VersionedFile.getLatestFile( core.getDataDir(), f );
-        Config cfg = new Config( core.getResourceLoader(), f, is, null );
+        Config cfg = new Config( core.getResourceLoader(), f, new InputSource(is), null );
         map = loadElevationMap( cfg );
         elevationCache.put( reader, map );
       }

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/IndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/IndexSchema.java?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/IndexSchema.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/IndexSchema.java Sun Feb 27 12:52:05 2011
@@ -27,6 +27,7 @@ import org.apache.solr.common.ResourceLo
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.DOMUtil;
+import org.apache.solr.common.util.SystemIdResolver;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.Config;
 import org.apache.solr.core.SolrResourceLoader;
@@ -38,6 +39,7 @@ import org.apache.solr.search.SolrQueryP
 import org.apache.solr.util.plugin.AbstractPluginLoader;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.w3c.dom.*;
+import org.xml.sax.InputSource;
 
 import javax.xml.xpath.XPath;
 import javax.xml.xpath.XPathConstants;
@@ -110,22 +112,17 @@ public final class IndexSchema {
    * By default, this follows the normal config path directory searching rules.
    * @see SolrResourceLoader#openResource
    */
-  public IndexSchema(SolrConfig solrConfig, String name, InputStream is) {
+  public IndexSchema(SolrConfig solrConfig, String name, InputSource is) {
     this.solrConfig = solrConfig;
     if (name == null)
       name = DEFAULT_SCHEMA_FILE;
     this.resourceName = name;
     loader = solrConfig.getResourceLoader();
-    InputStream lis = is;
-    if (lis == null)
-      lis = loader.openSchema(name);
-    readSchema(lis);
-    if (lis != is) {
-      try {
-        lis.close();
-      }
-      catch(IOException xio) {} // ignore
+    if (is == null) {
+      is = new InputSource(loader.openSchema(name));
+      is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
     }
+    readSchema(is);
     loader.inform( loader );
   }
 
@@ -414,7 +411,7 @@ public final class IndexSchema {
     }
   }
 
-  private void readSchema(InputStream is) {
+  private void readSchema(InputSource is) {
     log.info("Reading Solr Schema");
 
     try {

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/util/xslt/TransformerProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/util/xslt/TransformerProvider.java?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/util/xslt/TransformerProvider.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/util/xslt/TransformerProvider.java Sun Feb 27 12:52:05 2011
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.commons.io.IOUtils;
 
 import javax.xml.transform.Templates;
 import javax.xml.transform.Transformer;
@@ -29,6 +30,7 @@ import javax.xml.transform.TransformerFa
 import javax.xml.transform.stream.StreamSource;
 
 import org.apache.solr.common.ResourceLoader;
+import org.apache.solr.common.util.SystemIdResolver;
 import org.apache.solr.core.SolrConfig;
 
 /** Singleton that creates a Transformer for the XSLTServletFilter.
@@ -42,7 +44,6 @@ import org.apache.solr.core.SolrConfig;
 public class TransformerProvider {
   public static TransformerProvider instance = new TransformerProvider();
 
-  private final TransformerFactory tFactory = TransformerFactory.newInstance();
   private String lastFilename;
   private Templates lastTemplates = null;
   private long cacheExpires = 0;
@@ -99,8 +100,17 @@ public class TransformerProvider {
       if(log.isDebugEnabled()) {
         log.debug("compiling XSLT templates:" + filename);
       }
-      final InputStream xsltStream = loader.openResource("xslt/" + filename);
-      result = tFactory.newTemplates(new StreamSource(xsltStream));
+      final String fn = "xslt/" + filename;
+      final TransformerFactory tFactory = TransformerFactory.newInstance();
+      tFactory.setURIResolver(new SystemIdResolver(loader).asURIResolver());
+      final StreamSource src = new StreamSource(loader.openResource(fn),
+        SystemIdResolver.createSystemIdFromResourceName(fn));
+      try {
+        result = tFactory.newTemplates(src);
+      } finally {
+        // some XML parsers are broken and don't close the byte stream (but they should according to spec)
+        IOUtils.closeQuietly(src.getInputStream());
+      }
     } catch (Exception e) {
       log.error(getClass().getName(), "newTemplates", e);
       final IOException ioe = new IOException("Unable to initialize Templates '" + filename + "'");

Modified: lucene/dev/branches/branch_3x/solr/src/test-files/solr/conf/solrconfig-xinclude.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/test-files/solr/conf/solrconfig-xinclude.xml?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/test-files/solr/conf/solrconfig-xinclude.xml (original)
+++ lucene/dev/branches/branch_3x/solr/src/test-files/solr/conf/solrconfig-xinclude.xml Sun Feb 27 12:52:05 2011
@@ -245,10 +245,8 @@
   	<bool name="httpCaching">true</bool>
   </requestHandler>
 
-  <!--  FIXME: is there a way to make this test *not* depend on the CWD? -->
   <xi:include href="solrconfig-reqHandler.incl" xmlns:xi="http://www.w3.org/2001/XInclude"/>
 
-
   <requestHandler name="dismax" class="solr.DisMaxRequestHandler" >
     <lst name="defaults">
      <str name="q.alt">*:*</str>

Added: lucene/dev/branches/branch_3x/solr/src/test-files/solr/conf/xslt/dummy-using-include.xsl
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/test-files/solr/conf/xslt/dummy-using-include.xsl?rev=1075044&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/test-files/solr/conf/xslt/dummy-using-include.xsl (added)
+++ lucene/dev/branches/branch_3x/solr/src/test-files/solr/conf/xslt/dummy-using-include.xsl Sun Feb 27 12:52:05 2011
@@ -0,0 +1,31 @@
+<?xml version='1.0' encoding='UTF-8'?>
+
+<!-- 
+ * 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.
+ -->
+
+<!-- 
+
+
+Simple Dummy transform to demonstrate XSLTResponseWriter
+
+
+ -->
+<xsl:stylesheet version='1.0'
+    xmlns:xsl='http://www.w3.org/1999/XSL/Transform'
+>
+  <xsl:include href="dummy.xsl"/>
+</xsl:stylesheet>

Modified: lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/OutputWriterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/OutputWriterTest.java?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/OutputWriterTest.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/OutputWriterTest.java Sun Feb 27 12:52:05 2011
@@ -83,6 +83,15 @@ public class OutputWriterTest extends So
         assertTrue(out.contains("DUMMY"));
     }
     
+    @Test
+    public void testTrivialXsltWriterInclude() throws Exception {
+        lrf.args.put("wt", "xslt");
+        lrf.args.put("tr", "dummy-using-include.xsl");
+        String out = h.query(req("foo"));
+        // System.out.println(out);
+        assertTrue(out.contains("DUMMY"));
+    }
+    
     
     ////////////////////////////////////////////////////////////////////////////
     /** An output writer that doesn't do anything useful. */

Added: lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/common/util/TestSystemIdResolver.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/common/util/TestSystemIdResolver.java?rev=1075044&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/common/util/TestSystemIdResolver.java (added)
+++ lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/common/util/TestSystemIdResolver.java Sun Feb 27 12:52:05 2011
@@ -0,0 +1,72 @@
+/**
+ * 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.util;
+
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.common.ResourceLoader;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.SolrTestCaseJ4;
+
+import java.io.File;
+import org.xml.sax.InputSource;
+import org.apache.commons.io.IOUtils;
+
+public class TestSystemIdResolver extends LuceneTestCase {
+  
+  private void assertEntityResolving(SystemIdResolver resolver, String expectedSystemId, String base, String systemId) throws Exception {
+    final InputSource is = resolver.resolveEntity(null, null, base, systemId);
+    try {
+      assertEquals("Resolved SystemId does not match", expectedSystemId, is.getSystemId());
+    } finally {
+      IOUtils.closeQuietly(is.getByteStream());
+    }
+  }
+  
+  public void testResolving() throws Exception {
+    final ResourceLoader loader = new SolrResourceLoader(SolrTestCaseJ4.TEST_HOME, this.getClass().getClassLoader());
+    final SystemIdResolver resolver = new SystemIdResolver(loader);
+    final String fileUri = new File(SolrTestCaseJ4.TEST_HOME+"/crazy-path-to-config.xml").toURI().toASCIIString();
+    
+    assertEquals("solrres:/test.xml", SystemIdResolver.createSystemIdFromResourceName("test.xml"));
+    assertEquals("solrres://@/usr/local/etc/test.xml", SystemIdResolver.createSystemIdFromResourceName("/usr/local/etc/test.xml"));
+    assertEquals("solrres://@/test.xml", SystemIdResolver.createSystemIdFromResourceName(File.separatorChar+"test.xml"));
+    
+    // check relative URI resolving
+    assertEquals("solrres:/test.xml", resolver.resolveRelativeURI("solrres:/base.xml", "test.xml").toASCIIString());
+    assertEquals("solrres://@/etc/test.xml",
+      resolver.resolveRelativeURI("solrres://@/usr/local/etc/base.xml", "../../../etc/test.xml").toASCIIString());
+    // special case: if relative URI starts with "/" convert to an absolute solrres://@/-URI
+    assertEquals("solrres://@/a/test.xml", resolver.resolveRelativeURI("solrres:/base.xml", "/a/test.xml").toASCIIString());
+    // test, that resolving works if somebody uses an absolute file:-URI in a href attribute, it should be preserved
+    assertEquals(fileUri, resolver.resolveRelativeURI("solrres:/base.xml", fileUri).toASCIIString());
+    assertEquals("solrres:/base.xml", resolver.resolveRelativeURI(fileUri, "solrres:/base.xml").toASCIIString());
+    
+    // do some real resolves to I nputStreams with real existing files
+    assertEntityResolving(resolver, "solrres:/schema.xml", "solrres:/solrconfig.xml", "schema.xml");
+    assertEntityResolving(resolver, "solrres:/org/apache/solr/common/util/TestSystemIdResolver.class",
+      "solrres:/org/apache/solr/common/ResourceLoader.class", "util/TestSystemIdResolver.class");
+    assertEntityResolving(resolver, SystemIdResolver.createSystemIdFromResourceName(SolrTestCaseJ4.TEST_HOME+"/conf/schema.xml"),
+      SystemIdResolver.createSystemIdFromResourceName(SolrTestCaseJ4.TEST_HOME+"/conf/solrconfig.xml"), "schema.xml");
+    assertEntityResolving(resolver, SystemIdResolver.createSystemIdFromResourceName(SolrTestCaseJ4.TEST_HOME+"/crazy-path-to-schema.xml"),
+      SystemIdResolver.createSystemIdFromResourceName(SolrTestCaseJ4.TEST_HOME+"/crazy-path-to-config.xml"), "crazy-path-to-schema.xml");
+    
+    // test, that resolving works if somebody uses an absolute file:-URI in a href attribute, the resolver should return null (default fallback)
+    assertNull(resolver.resolveEntity(null, null, "solrres:/solrconfig.xml", fileUri));
+  }
+
+}

Modified: lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/core/TestXIncludeConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/core/TestXIncludeConfig.java?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/core/TestXIncludeConfig.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/core/TestXIncludeConfig.java Sun Feb 27 12:52:05 2011
@@ -17,11 +17,8 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import java.io.File;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.solr.util.AbstractSolrTestCase;
 import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.util.AbstractSolrTestCase;
 
 import javax.xml.parsers.DocumentBuilderFactory;
 
@@ -46,19 +43,16 @@ public class TestXIncludeConfig extends 
 
   @Override
   public void setUp() throws Exception {
-    File dest = new File("solrconfig-reqHandler.incl");
-    dest.deleteOnExit();
-    FileUtils.copyFile(getFile("solr/conf/solrconfig-reqHandler.incl"), dest);
     supports = true;
     javax.xml.parsers.DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     try {
       //see whether it even makes sense to run this test
       dbf.setXIncludeAware(true);
       dbf.setNamespaceAware(true);
-      super.setUp();
     } catch (UnsupportedOperationException e) {
       supports = false;
     }
+    super.setUp();
   }
 
   public void testXInclude() throws Exception {

Modified: lucene/dev/branches/branch_3x/solr/src/webapp/src/org/apache/solr/servlet/SolrDispatchFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/webapp/src/org/apache/solr/servlet/SolrDispatchFilter.java?rev=1075044&r1=1075043&r2=1075044&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/webapp/src/org/apache/solr/servlet/SolrDispatchFilter.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/webapp/src/org/apache/solr/servlet/SolrDispatchFilter.java Sun Feb 27 12:52:05 2011
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.WeakHashMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.xml.sax.InputSource;
 
 import javax.servlet.Filter;
 import javax.servlet.FilterChain;
@@ -65,7 +66,7 @@ public class SolrDispatchFilter implemen
 
   public SolrDispatchFilter() {
     try {
-      adminRequestParser = new SolrRequestParsers(new Config(null,"solr",new ByteArrayInputStream("<root/>".getBytes("UTF-8")),"") );
+      adminRequestParser = new SolrRequestParsers(new Config(null,"solr",new InputSource(new ByteArrayInputStream("<root/>".getBytes("UTF-8"))),"") );
     } catch (Exception e) {
       //unlikely
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,e);