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 2012/09/27 12:10:42 UTC

svn commit: r1390921 - in /lucene/dev/trunk/solr: ./ contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/ core/src/java/org/apache/solr/handler/loader/ core/src/ja...

Author: uschindler
Date: Thu Sep 27 10:10:42 2012
New Revision: 1390921

URL: http://svn.apache.org/viewvc?rev=1390921&view=rev
Log:
SOLR-3895, SOLR-3614: XML and XSLT UpdateRequestHandler should not try to resolve external entities; fix XML parsing in XPathEntityProcessor to correctly expand named entities, but ignore external entities

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/EmptyEntityResolver.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestXPathEntityProcessor.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/XsltUpdateRequestHandlerTest.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1390921&r1=1390920&r2=1390921&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Thu Sep 27 10:10:42 2012
@@ -44,6 +44,13 @@ Optimizations
 * SOLR-3788: Admin Cores UI should redirect to newly created core details
   (steffkes)
 
+* SOLR-3895: XML and XSLT UpdateRequestHandler should not try to resolve
+  external entities. This improves speed of loading e.g. XSL-transformed
+  XHTML documents.  (Martin Herfurt, uschindler, hossman)
+
+* SOLR-3614: Fix XML parsing in XPathEntityProcessor to correctly expand
+  named entities, but ignore external entities.  (uschindler, hossman)
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java?rev=1390921&r1=1390920&r2=1390921&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java Thu Sep 27 10:10:42 2012
@@ -17,6 +17,7 @@
 package org.apache.solr.handler.dataimport;
 
 import org.apache.solr.common.util.XMLErrorLogger;
+import org.apache.solr.util.EmptyEntityResolver;
 import javax.xml.stream.XMLInputFactory;
 import static javax.xml.stream.XMLStreamConstants.*;
 import javax.xml.stream.XMLStreamException;
@@ -625,10 +626,22 @@ public class XPathRecordReader {
   }
 
   static XMLInputFactory factory = XMLInputFactory.newInstance();
-  static{
-    factory.setProperty(XMLInputFactory.IS_VALIDATING , Boolean.FALSE); 
-    factory.setProperty(XMLInputFactory.SUPPORT_DTD , Boolean.FALSE);
+  static {
+    EmptyEntityResolver.configureXMLInputFactory(factory);
     factory.setXMLReporter(XMLLOG);
+    try {
+      // The java 1.6 bundled stax parser (sjsxp) does not currently have a thread-safe
+      // XMLInputFactory, as that implementation tries to cache and reuse the
+      // XMLStreamReader.  Setting the parser-specific "reuse-instance" property to false
+      // prevents this.
+      // All other known open-source stax parsers (and the bea ref impl)
+      // have thread-safe factories.
+      factory.setProperty("reuse-instance", Boolean.FALSE);
+    } catch (IllegalArgumentException ex) {
+      // Other implementations will likely throw this exception since "reuse-instance"
+      // isimplementation specific.
+      LOG.debug("Unable to set the 'reuse-instance' property for the input chain: " + factory);
+    }
   }
 
   /**Implement this interface to stream records as and when one is found.

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestXPathEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestXPathEntityProcessor.java?rev=1390921&r1=1390920&r2=1390921&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestXPathEntityProcessor.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestXPathEntityProcessor.java Thu Sep 27 10:10:42 2012
@@ -87,7 +87,11 @@ public class TestXPathEntityProcessor ex
         break;
       result.add(row);
     }
-    assertEquals(2, ((List)result.get(0).get("a")).size());
+    List l = (List)result.get(0).get("a");
+    assertEquals(3, l.size());
+    assertEquals("1", l.get(0));
+    assertEquals("2", l.get(1));
+    assertEquals("ü", l.get(2));
   }
 
   @Test
@@ -298,7 +302,7 @@ public class TestXPathEntityProcessor ex
           + "\t\t<price>9.90</price>\n"
           + "\t\t<year>1982</year>\n" + "\t</cd>\n" + "</catalog>\t";
 
-  private static final String testXml = "<?xml version=\"1.0\" encoding=\"UTF-8\"?><root><a>1</a><a>2</a></root>";
+  private static final String testXml = "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n<!DOCTYPE root [\n<!ENTITY uuml \"&#252;\" >\n]>\n<root><a>1</a><a>2</a><a>&uuml;</a></root>";
 
   private static final String testXmlFlatten = "<?xml version=\"1.0\" encoding=\"UTF-8\"?><root><a>1<b>B</b>2</a></root>";
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java?rev=1390921&r1=1390920&r2=1390921&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java Thu Sep 27 10:10:42 2012
@@ -37,10 +37,12 @@ import org.apache.solr.common.params.Upd
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.handler.RequestHandlerUtils;
 import org.apache.solr.handler.UpdateRequestHandler;
+import org.apache.solr.util.EmptyEntityResolver;
 import org.apache.commons.io.IOUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.InputSource;
+import org.xml.sax.XMLReader;
 
 import javax.xml.stream.XMLStreamReader;
 import javax.xml.stream.XMLStreamException;
@@ -52,6 +54,7 @@ import javax.xml.transform.TransformerEx
 import javax.xml.transform.dom.DOMResult;
 import javax.xml.transform.dom.DOMSource;
 import javax.xml.transform.sax.SAXSource;
+import javax.xml.parsers.SAXParserFactory;
 
 import java.io.ByteArrayInputStream;
 import java.io.InputStream;
@@ -72,10 +75,14 @@ public class XMLLoader extends ContentSt
   
   int xsltCacheLifetimeSeconds;
   XMLInputFactory inputFactory;
+  SAXParserFactory saxFactory;
 
   @Override
   public XMLLoader init(SolrParams args) {
+    // Init StAX parser:
     inputFactory = XMLInputFactory.newInstance();
+    EmptyEntityResolver.configureXMLInputFactory(inputFactory);
+    inputFactory.setXMLReporter(xmllog);
     try {
       // The java 1.6 bundled stax parser (sjsxp) does not currently have a thread-safe
       // XMLInputFactory, as that implementation tries to cache and reuse the
@@ -84,13 +91,16 @@ public class XMLLoader extends ContentSt
       // All other known open-source stax parsers (and the bea ref impl)
       // have thread-safe factories.
       inputFactory.setProperty("reuse-instance", Boolean.FALSE);
-    }
-    catch (IllegalArgumentException ex) {
+    } catch (IllegalArgumentException ex) {
       // Other implementations will likely throw this exception since "reuse-instance"
       // isimplementation specific.
       log.debug("Unable to set the 'reuse-instance' property for the input chain: " + inputFactory);
     }
-    inputFactory.setXMLReporter(xmllog);
+    
+    // Init SAX parser (for XSL):
+    saxFactory = SAXParserFactory.newInstance();
+    saxFactory.setNamespaceAware(true); // XSL needs this!
+    EmptyEntityResolver.configureSAXParserFactory(saxFactory);
     
     xsltCacheLifetimeSeconds = XSLT_CACHE_DEFAULT;
     if(args != null) {
@@ -113,7 +123,7 @@ public class XMLLoader extends ContentSt
 
     String tr = req.getParams().get(CommonParams.TR,null);
     if(tr!=null) {
-      Transformer t = getTransformer(tr,req);
+      final Transformer t = getTransformer(tr,req);
       final DOMResult result = new DOMResult();
       
       // first step: read XML and build DOM using Transformer (this is no overhead, as XSL always produces
@@ -122,14 +132,17 @@ public class XMLLoader extends ContentSt
         is = stream.getStream();
         final InputSource isrc = new InputSource(is);
         isrc.setEncoding(charset);
-        final SAXSource source = new SAXSource(isrc);
+        final XMLReader xmlr = saxFactory.newSAXParser().getXMLReader();
+        xmlr.setErrorHandler(xmllog);
+        xmlr.setEntityResolver(EmptyEntityResolver.SAX_INSTANCE);
+        final SAXSource source = new SAXSource(xmlr, isrc);
         t.transform(source, result);
       } catch(TransformerException te) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, te.getMessage(), te);
       } finally {
         IOUtils.closeQuietly(is);
       }
-      // second step feed the intermediate DOM tree into StAX parser:
+      // second step: feed the intermediate DOM tree into StAX parser:
       try {
         parser = inputFactory.createXMLStreamReader(new DOMSource(result.getNode()));
         this.processUpdate(req, processor, parser);

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/EmptyEntityResolver.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/EmptyEntityResolver.java?rev=1390921&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/EmptyEntityResolver.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/EmptyEntityResolver.java Thu Sep 27 10:10:42 2012
@@ -0,0 +1,99 @@
+package org.apache.solr.util;
+
+/*
+ * 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.
+ */
+
+import java.io.InputStream;
+import org.xml.sax.InputSource;
+import org.xml.sax.EntityResolver;
+import javax.xml.XMLConstants;
+import javax.xml.parsers.SAXParserFactory;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLResolver;
+import javax.xml.transform.Source;
+import javax.xml.transform.stream.StreamSource;
+
+import org.apache.commons.io.input.ClosedInputStream;
+
+/**
+ * This class provides several singletons of entity resolvers used by
+ * SAX and StAX in the Java API. This is needed to make secure
+ * XML parsers, that don't resolve external entities from untrusted sources.
+ * <p>This class also provides static methods to configure SAX and StAX
+ * parsers to be safe.
+ * <p>Parsers will get an empty, closed stream for every external
+ * entity, so they will not fail while parsing (unless the external entity
+ * is needed for processing!).
+ */
+public final class EmptyEntityResolver {
+
+  public static final EntityResolver SAX_INSTANCE = new EntityResolver() {
+    @Override
+    public InputSource resolveEntity(String publicId, String systemId) {
+      return new InputSource(ClosedInputStream.CLOSED_INPUT_STREAM);
+    }
+  };
+
+  public static final XMLResolver STAX_INSTANCE = new XMLResolver() {
+    @Override
+    public InputStream resolveEntity(String publicId, String systemId, String baseURI, String namespace) {
+      return ClosedInputStream.CLOSED_INPUT_STREAM;
+    }
+  };
+  
+  // no instance!
+  private EmptyEntityResolver() {}
+  
+  private static void trySetSAXFeature(SAXParserFactory saxFactory, String feature, boolean enabled) {
+    try {
+      saxFactory.setFeature(feature, enabled);
+    } catch (Exception ex) {
+      // ignore
+    }
+  }
+  
+  /** Configures the given {@link SAXParserFactory} to do secure XML processing of untrusted sources.
+   * It is required to also set {@link #SAX_INSTANCE} on the created {@link XMLReader}.
+   * @see #SAX_INSTANCE
+   */
+  public static void configureSAXParserFactory(SAXParserFactory saxFactory) {
+    // don't enable validation of DTDs:
+    saxFactory.setValidating(false);
+    // enable secure processing:
+    trySetSAXFeature(saxFactory, XMLConstants.FEATURE_SECURE_PROCESSING, true);
+  }
+  
+  private static void trySetStAXProperty(XMLInputFactory inputFactory, String key, Object value) {
+    try {
+      inputFactory.setProperty(key, value);
+    } catch (Exception ex) {
+      // ignore
+    }
+  }
+  
+  /** Configures the given {@link XMLInputFactory} to not parse external entities.
+   * No further configuration on is needed, all required entity resolvers are configured.
+   */
+  public static void configureXMLInputFactory(XMLInputFactory inputFactory) {
+    // don't enable validation of DTDs:
+    trySetStAXProperty(inputFactory, XMLInputFactory.IS_VALIDATING, Boolean.FALSE);
+    // enable this to *not* produce parsing failure on external entities:
+    trySetStAXProperty(inputFactory, XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, Boolean.TRUE);
+    inputFactory.setXMLResolver(EmptyEntityResolver.STAX_INSTANCE);
+  }
+  
+}

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java?rev=1390921&r1=1390920&r2=1390921&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java Thu Sep 27 10:10:42 2012
@@ -111,6 +111,46 @@ public class XmlUpdateRequestHandlerTest
   }
   
   @Test
+  public void testExternalEntities() throws Exception
+  {
+    String file = getFile("mailing_lists.pdf").toURI().toASCIIString();
+    String xml = 
+      "<?xml version=\"1.0\"?>" +
+      // check that external entities are not resolved!
+      "<!DOCTYPE foo [<!ENTITY bar SYSTEM \""+file+"\">]>" +
+      "<add>" +
+      "  &bar;" +
+      "  <doc>" +
+      "    <field name=\"id\">12345</field>" +
+      "    <field name=\"name\">kitten</field>" +
+      "  </doc>" +
+      "</add>";
+    SolrQueryRequest req = req();
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    BufferingRequestProcessor p = new BufferingRequestProcessor(null);
+    XMLLoader loader = new XMLLoader().init(null);
+    loader.load(req, rsp, new ContentStreamBase.StringStream(xml), p);
+
+    AddUpdateCommand add = p.addCommands.get(0);
+    assertEquals("12345", add.solrDoc.getField("id").getFirstValue());
+    req.close();
+  }
+
+  public void testNamedEntity() throws Exception {
+    assertU("<?xml version=\"1.0\" ?>\n"+
+            "<!DOCTYPE add [\n<!ENTITY wacky \"zzz\" >\n]>"+
+            "<add><doc>"+
+            "<field name=\"id\">1</field>"+
+            "<field name=\"foo_s\">&wacky;</field>" + 
+            "</doc></add>");
+    
+    assertU("<commit/>");
+    assertQ(req("foo_s:zzz"),
+            "//*[@numFound='1']"
+            );
+  }
+  
+  @Test
   public void testReadDelete() throws Exception {
       String xml =
         "<update>" +

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/XsltUpdateRequestHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/XsltUpdateRequestHandlerTest.java?rev=1390921&r1=1390920&r2=1390921&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/XsltUpdateRequestHandlerTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/XsltUpdateRequestHandlerTest.java Thu Sep 27 10:10:42 2012
@@ -28,9 +28,13 @@ import org.apache.solr.common.util.Conte
 import org.apache.solr.common.util.ContentStreamBase;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.loader.XMLLoader;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.response.QueryResponseWriter;
+import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.processor.BufferingRequestProcessor;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -89,4 +93,36 @@ public class XsltUpdateRequestHandlerTes
             , "//int[@name='id'][.='12345']"
         );
   }
+  
+  @Test
+  public void testEntities() throws Exception
+  {
+    // use a binary file, so when it's loaded fail with XML eror:
+    String file = getFile("mailing_lists.pdf").toURI().toASCIIString();
+    String xml = 
+      "<?xml version=\"1.0\"?>" +
+      "<!DOCTYPE foo [" + 
+      // check that external entities are not resolved!
+      "<!ENTITY bar SYSTEM \""+file+"\">"+
+      // but named entities should be
+      "<!ENTITY wacky \"zzz\">"+
+      "]>" +
+      "<random>" +
+      " &bar;" +
+      " <document>" +
+      "  <node name=\"id\" value=\"12345\"/>" +
+      "  <node name=\"foo_s\" value=\"&wacky;\"/>" +
+      " </document>" +
+      "</random>";
+    SolrQueryRequest req = req(CommonParams.TR, "xsl-update-handler-test.xsl");
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    BufferingRequestProcessor p = new BufferingRequestProcessor(null);
+    XMLLoader loader = new XMLLoader().init(null);
+    loader.load(req, rsp, new ContentStreamBase.StringStream(xml), p);
+
+    AddUpdateCommand add = p.addCommands.get(0);
+    assertEquals("12345", add.solrDoc.getField("id").getFirstValue());
+    assertEquals("zzz", add.solrDoc.getField("foo_s").getFirstValue());
+    req.close();
+  }  
 }