You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2022/10/07 09:48:15 UTC

[hadoop] branch branch-3.3 updated: HADOOP-18469. Add secure XML parser factories to XMLUtils (#4940)

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

stevel pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
     new 80781306dd6 HADOOP-18469. Add secure XML parser factories to XMLUtils (#4940)
80781306dd6 is described below

commit 80781306dd6c02666dc04b9131f4f7c677318adf
Author: Steve Loughran <st...@cloudera.com>
AuthorDate: Fri Oct 7 10:47:55 2022 +0100

    HADOOP-18469. Add secure XML parser factories to XMLUtils (#4940)
    
    
    Add to XMLUtils a set of methods to create secure XML Parsers/transformers,
    locking down DTD, schema, XXE exposure.
    
    Use these wherever XML parsers are created.
    
    Contributed by PJ Fanning
---
 .../java/org/apache/hadoop/conf/Configuration.java |  11 +-
 .../org/apache/hadoop/util/HostsFileReader.java    |   2 +-
 .../main/java/org/apache/hadoop/util/XMLUtils.java | 100 ++++++++++++++-
 .../java/org/apache/hadoop/cli/CLITestHelper.java  |   5 +-
 .../org/apache/hadoop/conf/TestConfServlet.java    |   6 +-
 .../java/org/apache/hadoop/util/TestXMLUtils.java  | 134 +++++++++++++++++++++
 .../src/test/resources/xml/entity-dtd.xml          |  22 ++++
 .../src/test/resources/xml/external-dtd.xml        |  23 ++++
 .../hadoop/tools/rumen/JobConfigurationParser.java |   4 +-
 .../hadoop/tools/rumen/ParsedConfigFile.java       |  18 ++-
 10 files changed, 302 insertions(+), 23 deletions(-)

diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index ad902de3045..d0c7cce75e9 100755
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -24,7 +24,6 @@ import com.ctc.wstx.io.SystemId;
 import com.ctc.wstx.stax.WstxInputFactory;
 import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonGenerator;
-import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 import java.io.BufferedInputStream;
 import java.io.DataInput;
@@ -87,6 +86,7 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Charsets;
 import org.apache.commons.collections.map.UnmodifiableMap;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -98,18 +98,19 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.alias.CredentialProvider;
 import org.apache.hadoop.security.alias.CredentialProvider.CredentialEntry;
 import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
+import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.XMLUtils;
+
 import org.codehaus.stax2.XMLStreamReader2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
-
 import static org.apache.commons.lang3.StringUtils.isBlank;
 import static org.apache.commons.lang3.StringUtils.isNotBlank;
 
@@ -3580,7 +3581,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     try {
       DOMSource source = new DOMSource(doc);
       StreamResult result = new StreamResult(out);
-      TransformerFactory transFactory = TransformerFactory.newInstance();
+      TransformerFactory transFactory = XMLUtils.newSecureTransformerFactory();
       Transformer transformer = transFactory.newTransformer();
 
       // Important to not hold Configuration log while writing result, since
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java
index 5141740a3d2..054f6235fe9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java
@@ -147,8 +147,8 @@ public class HostsFileReader {
       String filename, InputStream fileInputStream, Map<String, Integer> map)
           throws IOException {
     Document dom;
-    DocumentBuilderFactory builder = DocumentBuilderFactory.newInstance();
     try {
+      DocumentBuilderFactory builder = XMLUtils.newSecureDocumentBuilderFactory();
       DocumentBuilder db = builder.newDocumentBuilder();
       dom = db.parse(fileInputStream);
       // Examples:
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/XMLUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/XMLUtils.java
index e2b9e414ad3..0b5084f5f78 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/XMLUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/XMLUtils.java
@@ -18,12 +18,19 @@
 
 package org.apache.hadoop.util;
 
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.parsers.SAXParserFactory;
 import javax.xml.transform.*;
+import javax.xml.transform.sax.SAXTransformerFactory;
 import javax.xml.transform.stream.*;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+import org.xml.sax.SAXException;
+
 import java.io.*;
 
 /**
@@ -33,6 +40,19 @@ import java.io.*;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class XMLUtils {
+
+  private static final String DISALLOW_DOCTYPE_DECL =
+      "http://apache.org/xml/features/disallow-doctype-decl";
+  private static final String LOAD_EXTERNAL_DECL =
+      "http://apache.org/xml/features/nonvalidating/load-external-dtd";
+  private static final String EXTERNAL_GENERAL_ENTITIES =
+      "http://xml.org/sax/features/external-general-entities";
+  private static final String EXTERNAL_PARAMETER_ENTITIES =
+      "http://xml.org/sax/features/external-parameter-entities";
+  private static final String CREATE_ENTITY_REF_NODES =
+      "http://apache.org/xml/features/dom/create-entity-ref-nodes";
+
+
   /**
    * Transform input xml given a stylesheet.
    * 
@@ -49,7 +69,7 @@ public class XMLUtils {
                                ) 
     throws TransformerConfigurationException, TransformerException {
     // Instantiate a TransformerFactory
-    TransformerFactory tFactory = TransformerFactory.newInstance();
+    TransformerFactory tFactory = newSecureTransformerFactory();
 
     // Use the TransformerFactory to process the  
     // stylesheet and generate a Transformer
@@ -61,4 +81,82 @@ public class XMLUtils {
     // and send the output to a Result object.
     transformer.transform(new StreamSource(xml), new StreamResult(out));
   }
+
+  /**
+   * This method should be used if you need a {@link DocumentBuilderFactory}. Use this method
+   * instead of {@link DocumentBuilderFactory#newInstance()}. The factory that is returned has
+   * secure configuration enabled.
+   *
+   * @return a {@link DocumentBuilderFactory} with secure configuration enabled
+   * @throws ParserConfigurationException if the {@code JAXP} parser does not support the
+   * secure configuration
+   */
+  public static DocumentBuilderFactory newSecureDocumentBuilderFactory()
+          throws ParserConfigurationException {
+    DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+    dbf.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true);
+    dbf.setFeature(DISALLOW_DOCTYPE_DECL, true);
+    dbf.setFeature(LOAD_EXTERNAL_DECL, false);
+    dbf.setFeature(EXTERNAL_GENERAL_ENTITIES, false);
+    dbf.setFeature(EXTERNAL_PARAMETER_ENTITIES, false);
+    dbf.setFeature(CREATE_ENTITY_REF_NODES, false);
+    return dbf;
+  }
+
+  /**
+   * This method should be used if you need a {@link SAXParserFactory}. Use this method
+   * instead of {@link SAXParserFactory#newInstance()}. The factory that is returned has
+   * secure configuration enabled.
+   *
+   * @return a {@link SAXParserFactory} with secure configuration enabled
+   * @throws ParserConfigurationException if the {@code JAXP} parser does not support the
+   * secure configuration
+   * @throws SAXException if there are another issues when creating the factory
+   */
+  public static SAXParserFactory newSecureSAXParserFactory()
+          throws SAXException, ParserConfigurationException {
+    SAXParserFactory spf = SAXParserFactory.newInstance();
+    spf.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true);
+    spf.setFeature(DISALLOW_DOCTYPE_DECL, true);
+    spf.setFeature(LOAD_EXTERNAL_DECL, false);
+    spf.setFeature(EXTERNAL_GENERAL_ENTITIES, false);
+    spf.setFeature(EXTERNAL_PARAMETER_ENTITIES, false);
+    return spf;
+  }
+
+  /**
+   * This method should be used if you need a {@link TransformerFactory}. Use this method
+   * instead of {@link TransformerFactory#newInstance()}. The factory that is returned has
+   * secure configuration enabled.
+   *
+   * @return a {@link TransformerFactory} with secure configuration enabled
+   * @throws TransformerConfigurationException if the {@code JAXP} transformer does not
+   * support the secure configuration
+   */
+  public static TransformerFactory newSecureTransformerFactory()
+          throws TransformerConfigurationException {
+    TransformerFactory trfactory = TransformerFactory.newInstance();
+    trfactory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true);
+    trfactory.setAttribute(XMLConstants.ACCESS_EXTERNAL_DTD, "");
+    trfactory.setAttribute(XMLConstants.ACCESS_EXTERNAL_STYLESHEET, "");
+    return trfactory;
+  }
+
+  /**
+   * This method should be used if you need a {@link SAXTransformerFactory}. Use this method
+   * instead of {@link SAXTransformerFactory#newInstance()}. The factory that is returned has
+   * secure configuration enabled.
+   *
+   * @return a {@link SAXTransformerFactory} with secure configuration enabled
+   * @throws TransformerConfigurationException if the {@code JAXP} transformer does not
+   * support the secure configuration
+   */
+  public static SAXTransformerFactory newSecureSAXTransformerFactory()
+          throws TransformerConfigurationException {
+    SAXTransformerFactory trfactory = (SAXTransformerFactory) SAXTransformerFactory.newInstance();
+    trfactory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true);
+    trfactory.setAttribute(XMLConstants.ACCESS_EXTERNAL_DTD, "");
+    trfactory.setAttribute(XMLConstants.ACCESS_EXTERNAL_STYLESHEET, "");
+    return trfactory;
+  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
index ada4cd80e48..f80c62535a1 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.XMLUtils;
+
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -34,7 +36,6 @@ import org.xml.sax.SAXException;
 import org.xml.sax.helpers.DefaultHandler;
 
 import javax.xml.parsers.SAXParser;
-import javax.xml.parsers.SAXParserFactory;
 import java.io.File;
 import java.util.ArrayList;
 
@@ -76,7 +77,7 @@ public class CLITestHelper {
       boolean success = false;
       testConfigFile = TEST_CACHE_DATA_DIR + File.separator + testConfigFile;
       try {
-        SAXParser p = (SAXParserFactory.newInstance()).newSAXParser();
+        SAXParser p = XMLUtils.newSecureSAXParserFactory().newSAXParser();
         p.parse(testConfigFile, getConfigParser());
         success = true;
       } catch (Exception e) {
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java
index eae9a1fc744..5809aa4da5b 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java
@@ -41,9 +41,12 @@ import org.xml.sax.InputSource;
 import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
 
 import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.util.XMLUtils;
+
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
+
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.mock;
 import static org.junit.Assert.*;
@@ -217,8 +220,7 @@ public class TestConfServlet {
     ConfServlet.writeResponse(getTestConf(), sw, "xml");
     String xml = sw.toString();
 
-    DocumentBuilderFactory docBuilderFactory 
-      = DocumentBuilderFactory.newInstance();
+    DocumentBuilderFactory docBuilderFactory = XMLUtils.newSecureDocumentBuilderFactory();
     DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
     Document doc = builder.parse(new InputSource(new StringReader(xml)));
     NodeList nameNodes = doc.getElementsByTagName("name");
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestXMLUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestXMLUtils.java
new file mode 100644
index 00000000000..ec1b74338a1
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestXMLUtils.java
@@ -0,0 +1,134 @@
+/**
+ * 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.hadoop.util;
+
+import java.io.InputStream;
+import java.io.StringReader;
+import java.io.StringWriter;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.SAXParser;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+import javax.xml.transform.stream.StreamSource;
+
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.w3c.dom.Document;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
+
+public class TestXMLUtils extends AbstractHadoopTestBase {
+
+  @Test
+  public void testSecureDocumentBuilderFactory() throws Exception {
+    DocumentBuilder db = XMLUtils.newSecureDocumentBuilderFactory().newDocumentBuilder();
+    Document doc = db.parse(new InputSource(new StringReader("<root/>")));
+    Assertions.assertThat(doc).describedAs("parsed document").isNotNull();
+  }
+
+  @Test(expected = SAXException.class)
+  public void testExternalDtdWithSecureDocumentBuilderFactory() throws Exception {
+    DocumentBuilder db = XMLUtils.newSecureDocumentBuilderFactory().newDocumentBuilder();
+    try (InputStream stream = getResourceStream("/xml/external-dtd.xml")) {
+      Document doc = db.parse(stream);
+    }
+  }
+
+  @Test(expected = SAXException.class)
+  public void testEntityDtdWithSecureDocumentBuilderFactory() throws Exception {
+    DocumentBuilder db = XMLUtils.newSecureDocumentBuilderFactory().newDocumentBuilder();
+    try (InputStream stream = getResourceStream("/xml/entity-dtd.xml")) {
+      Document doc = db.parse(stream);
+    }
+  }
+
+  @Test
+  public void testSecureSAXParserFactory() throws Exception {
+    SAXParser parser = XMLUtils.newSecureSAXParserFactory().newSAXParser();
+    parser.parse(new InputSource(new StringReader("<root/>")), new DefaultHandler());
+  }
+
+  @Test(expected = SAXException.class)
+  public void testExternalDtdWithSecureSAXParserFactory() throws Exception {
+    SAXParser parser = XMLUtils.newSecureSAXParserFactory().newSAXParser();
+    try (InputStream stream = getResourceStream("/xml/external-dtd.xml")) {
+      parser.parse(stream, new DefaultHandler());
+    }
+  }
+
+  @Test(expected = SAXException.class)
+  public void testEntityDtdWithSecureSAXParserFactory() throws Exception {
+    SAXParser parser = XMLUtils.newSecureSAXParserFactory().newSAXParser();
+    try (InputStream stream = getResourceStream("/xml/entity-dtd.xml")) {
+      parser.parse(stream, new DefaultHandler());
+    }
+  }
+
+  @Test
+  public void testSecureTransformerFactory() throws Exception {
+    Transformer transformer = XMLUtils.newSecureTransformerFactory().newTransformer();
+    DocumentBuilder db = XMLUtils.newSecureDocumentBuilderFactory().newDocumentBuilder();
+    Document doc = db.parse(new InputSource(new StringReader("<root/>")));
+    try (StringWriter stringWriter = new StringWriter()) {
+      transformer.transform(new DOMSource(doc), new StreamResult(stringWriter));
+      Assertions.assertThat(stringWriter.toString()).contains("<root");
+    }
+  }
+
+  @Test(expected = TransformerException.class)
+  public void testExternalDtdWithSecureTransformerFactory() throws Exception {
+    Transformer transformer = XMLUtils.newSecureTransformerFactory().newTransformer();
+    try (
+        InputStream stream = getResourceStream("/xml/external-dtd.xml");
+        StringWriter stringWriter = new StringWriter()
+    ) {
+      transformer.transform(new StreamSource(stream), new StreamResult(stringWriter));
+    }
+  }
+
+  @Test
+  public void testSecureSAXTransformerFactory() throws Exception {
+    Transformer transformer = XMLUtils.newSecureSAXTransformerFactory().newTransformer();
+    DocumentBuilder db = XMLUtils.newSecureDocumentBuilderFactory().newDocumentBuilder();
+    Document doc = db.parse(new InputSource(new StringReader("<root/>")));
+    try (StringWriter stringWriter = new StringWriter()) {
+      transformer.transform(new DOMSource(doc), new StreamResult(stringWriter));
+      Assertions.assertThat(stringWriter.toString()).contains("<root");
+    }
+  }
+
+  @Test(expected = TransformerException.class)
+  public void testExternalDtdWithSecureSAXTransformerFactory() throws Exception {
+    Transformer transformer = XMLUtils.newSecureSAXTransformerFactory().newTransformer();
+    try (
+        InputStream stream = getResourceStream("/xml/external-dtd.xml");
+        StringWriter stringWriter = new StringWriter()
+    ) {
+      transformer.transform(new StreamSource(stream), new StreamResult(stringWriter));
+    }
+  }
+
+  private static InputStream getResourceStream(final String filename) {
+    return TestXMLUtils.class.getResourceAsStream(filename);
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/xml/entity-dtd.xml b/hadoop-common-project/hadoop-common/src/test/resources/xml/entity-dtd.xml
new file mode 100644
index 00000000000..a3926bd67ad
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/resources/xml/entity-dtd.xml
@@ -0,0 +1,22 @@
+<?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.
+-->
+<!DOCTYPE lolz [
+        <!ENTITY lol "lol">
+        <!ELEMENT lolz (#PCDATA)>
+        ]>
+<lolz>&lol;</lolz>
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/xml/external-dtd.xml b/hadoop-common-project/hadoop-common/src/test/resources/xml/external-dtd.xml
new file mode 100644
index 00000000000..08a13938f5f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/resources/xml/external-dtd.xml
@@ -0,0 +1,23 @@
+<?xml version = "1.0" encoding = "UTF-8" standalone = "no" ?>
+<!--
+   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.
+-->
+<!DOCTYPE address SYSTEM "address.dtd">
+<address>
+    <name>First Last</name>
+    <company>Acme</company>
+    <phone>(555) 123-4567</phone>
+</address>
diff --git a/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/JobConfigurationParser.java b/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/JobConfigurationParser.java
index 7e79179721f..9cd2f4778fc 100644
--- a/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/JobConfigurationParser.java
+++ b/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/JobConfigurationParser.java
@@ -25,6 +25,8 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
 
+import org.apache.hadoop.util.XMLUtils;
+
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
@@ -55,7 +57,7 @@ public class JobConfigurationParser {
     Properties result = new Properties();
 
     try {
-      DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+      DocumentBuilderFactory dbf = XMLUtils.newSecureDocumentBuilderFactory();
 
       DocumentBuilder db = dbf.newDocumentBuilder();
 
diff --git a/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/ParsedConfigFile.java b/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/ParsedConfigFile.java
index 1d85872c08d..a6c8bdad87d 100644
--- a/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/ParsedConfigFile.java
+++ b/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/ParsedConfigFile.java
@@ -17,28 +17,27 @@
  */
 package org.apache.hadoop.tools.rumen;
 
+import java.io.IOException;
+import java.io.StringReader;
 import java.util.Properties;
 import java.util.regex.Pattern;
 import java.util.regex.Matcher;
 
-import java.io.InputStream;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-
-import java.nio.charset.Charset;
-
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.ParserConfigurationException;
 
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.util.XMLUtils;
+
 import org.w3c.dom.Document;
 import org.w3c.dom.NodeList;
 import org.w3c.dom.Node;
 import org.w3c.dom.Element;
 import org.w3c.dom.Text;
 
+import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
 
 class ParsedConfigFile {
@@ -46,7 +45,6 @@ class ParsedConfigFile {
       Pattern.compile("_(job_[0-9]+_[0-9]+)_");
   private static final Pattern heapPattern =
       Pattern.compile("-Xmx([0-9]+)([mMgG])");
-  private static final Charset UTF_8 = Charset.forName("UTF-8");
 
   final int heapMegabytes;
 
@@ -103,13 +101,11 @@ class ParsedConfigFile {
     }
 
     try {
-      InputStream is = new ByteArrayInputStream(xmlString.getBytes(UTF_8));
-
-      DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+      DocumentBuilderFactory dbf = XMLUtils.newSecureDocumentBuilderFactory();
 
       DocumentBuilder db = dbf.newDocumentBuilder();
 
-      Document doc = db.parse(is);
+      Document doc = db.parse(new InputSource(new StringReader(xmlString)));
 
       Element root = doc.getDocumentElement();
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org