You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@daffodil.apache.org by ol...@apache.org on 2021/01/21 14:39:02 UTC

[incubator-daffodil] branch master updated: SAX API Refactoring

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

olabusayo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-daffodil.git


The following commit(s) were added to refs/heads/master by this push:
     new b896c31  SAX API Refactoring
b896c31 is described below

commit b896c31a55c13178f202f68731c9baa930d0c493
Author: olabusayoT <50...@users.noreply.github.com>
AuthorDate: Mon Nov 16 10:56:37 2020 -0500

    SAX API Refactoring
    
    - moved inner classes into classes of their own
    - some refactoring to remove duplicate code
    
    DAFFODIL-2422
---
 .../src/main/scala/org/apache/daffodil/Main.scala  |  14 +-
 .../daffodil/processor/TestSAXParseAPI.scala       |   2 +-
 .../processor/TestSAXParseUnparseAPI.scala         |   2 +-
 .../daffodil/infoset/SAXInfosetOutputter.scala     | 115 ------------
 .../DaffodilParseOutputStreamContentHandler.scala  | 139 ++++++++++++++
 .../processors/DaffodilParseXMLReader.scala        | 209 +++++++++++++++++++++
 .../apache/daffodil/processors/DataProcessor.scala | 185 +-----------------
 .../tdml/processor/DaffodilTDMLDFDLProcessor.scala |   2 +-
 8 files changed, 356 insertions(+), 312 deletions(-)

diff --git a/daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala b/daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala
index 5b6f18e..f2c047b 100644
--- a/daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala
+++ b/daffodil-cli/src/main/scala/org/apache/daffodil/Main.scala
@@ -44,10 +44,8 @@ import javax.xml.parsers.SAXParserFactory
 import javax.xml.transform.TransformerFactory
 import javax.xml.transform.dom.DOMSource
 import javax.xml.transform.stream.StreamResult
-
 import org.apache.commons.io.IOUtils
 import org.apache.commons.io.output.NullOutputStream
-
 import org.apache.daffodil.api.DFDL
 import org.apache.daffodil.api.DFDL.DaffodilUnparseErrorSAXException
 import org.apache.daffodil.api.DFDL.ParseResult
@@ -69,7 +67,6 @@ import org.apache.daffodil.exceptions.UnsuppressableException
 import org.apache.daffodil.externalvars.Binding
 import org.apache.daffodil.externalvars.BindingException
 import org.apache.daffodil.externalvars.ExternalVariablesLoader
-import org.apache.daffodil.infoset.DaffodilParseOutputStreamContentHandler
 import org.apache.daffodil.infoset.InfosetInputter
 import org.apache.daffodil.infoset.InfosetOutputter
 import org.apache.daffodil.infoset.JDOMInfosetInputter
@@ -86,6 +83,7 @@ import org.apache.daffodil.infoset.XMLTextInfosetOutputter
 import org.apache.daffodil.io.DataDumper
 import org.apache.daffodil.io.FormatInfo
 import org.apache.daffodil.io.InputSourceDataInputStream
+import org.apache.daffodil.processors.DaffodilParseOutputStreamContentHandler
 import org.apache.daffodil.processors.DataLoc
 import org.apache.daffodil.processors.DataProcessor
 import org.apache.daffodil.processors.HasSetDebugger
@@ -783,11 +781,7 @@ object Main extends Logging {
    */
   def infosetDataToInputterData(infosetType: String, data: Either[Array[Byte],InputStream]): AnyRef = {
     infosetType match {
-      case "xml" => data match {
-        case Left(bytes) => bytes
-        case Right(is) => is
-      }
-      case "json" => data match {
+      case "xml" | "json" | "sax" => data match {
         case Left(bytes) => bytes
         case Right(is) => is
       }
@@ -819,10 +813,6 @@ object Main extends Logging {
           }
         }
       }
-      case "sax" => data match {
-        case Left(bytes) => bytes
-        case Right(is) => is
-      }
     }
   }
 
diff --git a/daffodil-core/src/test/scala/org/apache/daffodil/processor/TestSAXParseAPI.scala b/daffodil-core/src/test/scala/org/apache/daffodil/processor/TestSAXParseAPI.scala
index 86497d5..2e15aa0 100644
--- a/daffodil-core/src/test/scala/org/apache/daffodil/processor/TestSAXParseAPI.scala
+++ b/daffodil-core/src/test/scala/org/apache/daffodil/processor/TestSAXParseAPI.scala
@@ -24,7 +24,7 @@ import java.io.IOException
 import scala.xml.SAXParseException
 
 import org.apache.daffodil.Implicits.intercept
-import org.apache.daffodil.infoset.DaffodilParseOutputStreamContentHandler
+import org.apache.daffodil.processors.DaffodilParseOutputStreamContentHandler
 import org.apache.daffodil.processors.ParseResult
 import org.apache.daffodil.xml.XMLUtils
 import org.jdom2.input.sax.BuilderErrorHandler
diff --git a/daffodil-core/src/test/scala/org/apache/daffodil/processor/TestSAXParseUnparseAPI.scala b/daffodil-core/src/test/scala/org/apache/daffodil/processor/TestSAXParseUnparseAPI.scala
index 388fc5f..347bbe2 100644
--- a/daffodil-core/src/test/scala/org/apache/daffodil/processor/TestSAXParseUnparseAPI.scala
+++ b/daffodil-core/src/test/scala/org/apache/daffodil/processor/TestSAXParseUnparseAPI.scala
@@ -24,10 +24,10 @@ import scala.xml.Elem
 
 import javax.xml.parsers.SAXParserFactory
 import org.apache.daffodil.compiler.Compiler
-import org.apache.daffodil.infoset.DaffodilParseOutputStreamContentHandler
 import org.apache.daffodil.infoset.ScalaXMLInfosetInputter
 import org.apache.daffodil.infoset.ScalaXMLInfosetOutputter
 import org.apache.daffodil.io.InputSourceDataInputStream
+import org.apache.daffodil.processors.DaffodilParseOutputStreamContentHandler
 import org.apache.daffodil.processors.DataProcessor
 import org.apache.daffodil.processors.ParseResult
 import org.apache.daffodil.util.SchemaUtils
diff --git a/daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/SAXInfosetOutputter.scala b/daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/SAXInfosetOutputter.scala
index 61f5214..d2be4ba 100644
--- a/daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/SAXInfosetOutputter.scala
+++ b/daffodil-runtime1/src/main/scala/org/apache/daffodil/infoset/SAXInfosetOutputter.scala
@@ -17,19 +17,10 @@
 
 package org.apache.daffodil.infoset
 
-import java.io.OutputStream
-import java.io.OutputStreamWriter
-
-import scala.xml.NamespaceBinding
-
 import org.apache.daffodil.api.DFDL
 import org.apache.daffodil.dpath.NodeInfo
-import org.apache.daffodil.util.Indentable
-import org.apache.daffodil.util.MStackOfBoolean
 import org.apache.daffodil.xml.XMLUtils
-import org.xml.sax.Attributes
 import org.xml.sax.ContentHandler
-import org.xml.sax.Locator
 import org.xml.sax.SAXException
 import org.xml.sax.helpers.AttributesImpl
 
@@ -218,110 +209,4 @@ class SAXInfosetOutputter(xmlReader: DFDL.DaffodilParseXMLReader)
 
 }
 
-class DaffodilParseOutputStreamContentHandler(out: OutputStream, pretty: Boolean = false)
-  extends ContentHandler with Indentable with XMLInfosetOutputter {
-  private val writer = new OutputStreamWriter(out)
-  private var prefixMapping: NamespaceBinding = null
-  private val outputNewlineStack: MStackOfBoolean = {
-    val s = MStackOfBoolean()
-    s.push(false)
-    s
-  }
 
-  // if the top of the stack is true, we have guessed we should output a newline
-  private def outputNewline: Boolean = outputNewlineStack.top
-
-  def reset(): Unit = {
-    resetIndentation()
-    writer.flush()
-    prefixMapping = null
-    outputNewlineStack.clear()
-    outputNewlineStack.push(false) //to match initialization state
-    out.flush()
-  }
-
-  override def setDocumentLocator(locator: Locator): Unit = {
-    // do nothing
-  }
-
-  override def startDocument(): Unit = {
-    writer.write("""<?xml version="1.0" encoding="UTF-8"?>""")
-  }
-
-  override def endDocument(): Unit = {
-    writer.write(System.lineSeparator())
-    writer.flush()
-  }
-
-  override def startPrefixMapping(prefix: String, uri: String): Unit = {
-    val _prefix = if (prefix == "") null else prefix
-    prefixMapping = NamespaceBinding(_prefix, uri, prefixMapping)
-  }
-
-  override def endPrefixMapping(prefix: String): Unit = {
-  // do nothing
-  }
-
-  override def startElement(uri: String, localName: String, qName: String, atts: Attributes): Unit = {
-    // the pop/true removes whatever is on the stack which is our previous guess for whether we
-    // would need a newline after the previous end tag. As we are currently at the start of a new
-    // tag, we want to correct that assumption (in case it was false)
-    outputNewlineStack.pop()
-    outputNewlineStack.push(true)
-    if (pretty) {
-      writer.write(System.lineSeparator())
-      outputIndentation(writer)
-    }
-    // handle start of tag
-    writer.write("<")
-    writer.write(qName)
-    // handle attributes
-    for (i <- 0 until atts.getLength) {
-      val attsValue = atts.getValue(i)
-      val attsQName = atts.getQName(i)
-      writer.write(s""" $attsQName="$attsValue"""")
-    }
-    // handle namespaces
-    if (prefixMapping != null) {
-      val pm = prefixMapping.toString()
-      writer.write(pm)
-      prefixMapping = null
-    }
-    // handle end of tag
-    writer.write(">")
-    incrementIndentation()
-    // this push makes the assumption that we would not need to output a newline after this end
-    // tag is complete
-    outputNewlineStack.push(false)
-  }
-
-  override def endElement(uri: String, localName: String, qName: String): Unit = {
-    decrementIndentation()
-    if (outputNewline) {
-      if (pretty) {
-        writer.write(System.lineSeparator())
-        outputIndentation(writer)
-      }
-    }
-    writer.write("</")
-    writer.write(qName)
-    writer.write(">")
-    outputNewlineStack.pop()
-  }
-
-  override def characters(ch: Array[Char], start: Int, length: Int): Unit = {
-    writer.write(ch, start, length)
-  }
-
-  override def ignorableWhitespace(ch: Array[Char], start: Int, length: Int): Unit = {
-    // do nothing
-  }
-
-  override def processingInstruction(target: String, data: String): Unit = {
-    // do nothing
-  }
-
-  override def skippedEntity(name: String): Unit = {
-    // do nothing
-  }
-}
diff --git a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DaffodilParseOutputStreamContentHandler.scala b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DaffodilParseOutputStreamContentHandler.scala
new file mode 100644
index 0000000..3255c03
--- /dev/null
+++ b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DaffodilParseOutputStreamContentHandler.scala
@@ -0,0 +1,139 @@
+/*
+ * 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.daffodil.processors
+
+import java.io.OutputStream
+import java.io.OutputStreamWriter
+
+import scala.xml.NamespaceBinding
+
+import org.apache.daffodil.infoset.XMLInfosetOutputter
+import org.apache.daffodil.util.Indentable
+import org.apache.daffodil.util.MStackOfBoolean
+import org.xml.sax.Attributes
+import org.xml.sax.ContentHandler
+import org.xml.sax.Locator
+
+class DaffodilParseOutputStreamContentHandler(out: OutputStream, pretty: Boolean = false)
+  extends ContentHandler with Indentable with XMLInfosetOutputter {
+  private val writer = new OutputStreamWriter(out)
+  private var prefixMapping: NamespaceBinding = null
+  private val outputNewlineStack: MStackOfBoolean = {
+    val s = MStackOfBoolean()
+    s.push(false)
+    s
+  }
+
+  // if the top of the stack is true, we have guessed we should output a newline
+  private def outputNewline: Boolean = outputNewlineStack.top
+
+  def reset(): Unit = {
+    resetIndentation()
+    writer.flush()
+    prefixMapping = null
+    outputNewlineStack.clear()
+    outputNewlineStack.push(false) //to match initialization state
+    out.flush()
+  }
+
+  override def setDocumentLocator(locator: Locator): Unit = {
+    // do nothing
+  }
+
+  override def startDocument(): Unit = {
+    writer.write("""<?xml version="1.0" encoding="UTF-8"?>""")
+  }
+
+  override def endDocument(): Unit = {
+    writer.write(System.lineSeparator())
+    writer.flush()
+  }
+
+  override def startPrefixMapping(prefix: String, uri: String): Unit = {
+    val _prefix = if (prefix == "") null else prefix
+    prefixMapping = NamespaceBinding(_prefix, uri, prefixMapping)
+  }
+
+  override def endPrefixMapping(prefix: String): Unit = {
+    // do nothing
+  }
+
+  override def startElement(
+    uri: String, localName: String, qName: String, atts: Attributes): Unit = {
+    // the pop/true removes whatever is on the stack which is our previous guess for whether we
+    // would need a newline after the previous end tag. As we are currently at the start of a new
+    // tag, we want to correct that assumption (in case it was false)
+    outputNewlineStack.pop()
+    outputNewlineStack.push(true)
+    if (pretty) {
+      writer.write(System.lineSeparator())
+      outputIndentation(writer)
+    }
+    // handle start of tag
+    writer.write("<")
+    writer.write(qName)
+    // handle attributes
+    for (i <- 0 until atts.getLength) {
+      val attsValue = atts.getValue(i)
+      val attsQName = atts.getQName(i)
+      writer.write(s""" $attsQName="$attsValue"""")
+    }
+    // handle namespaces
+    if (prefixMapping != null) {
+      val pm = prefixMapping.toString()
+      writer.write(pm)
+      prefixMapping = null
+    }
+    // handle end of tag
+    writer.write(">")
+    incrementIndentation()
+    // this push makes the assumption that we would not need to output a newline after this end
+    // tag is complete
+    outputNewlineStack.push(false)
+  }
+
+  override def endElement(uri: String, localName: String, qName: String): Unit = {
+    decrementIndentation()
+    if (outputNewline) {
+      if (pretty) {
+        writer.write(System.lineSeparator())
+        outputIndentation(writer)
+      }
+    }
+    writer.write("</")
+    writer.write(qName)
+    writer.write(">")
+    outputNewlineStack.pop()
+  }
+
+  override def characters(ch: Array[Char], start: Int, length: Int): Unit = {
+    writer.write(ch, start, length)
+  }
+
+  override def ignorableWhitespace(ch: Array[Char], start: Int, length: Int): Unit = {
+    // do nothing
+  }
+
+  override def processingInstruction(target: String, data: String): Unit = {
+    // do nothing
+  }
+
+  override def skippedEntity(name: String): Unit = {
+    // do nothing
+  }
+}
diff --git a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DaffodilParseXMLReader.scala b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DaffodilParseXMLReader.scala
new file mode 100644
index 0000000..71c98af
--- /dev/null
+++ b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DaffodilParseXMLReader.scala
@@ -0,0 +1,209 @@
+/*
+ * 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.daffodil.processors
+
+import java.io.IOException
+import java.io.InputStream
+import java.nio.file.Path
+import java.nio.file.Paths
+
+import scala.collection.mutable
+
+import org.apache.daffodil.api.DFDL
+import org.apache.daffodil.exceptions.SchemaFileLocation
+import org.apache.daffodil.infoset.SAXInfosetOutputter
+import org.apache.daffodil.io.InputSourceDataInputStream
+import org.apache.daffodil.xml.XMLUtils
+import org.xml.sax.ContentHandler
+import org.xml.sax.DTDHandler
+import org.xml.sax.EntityResolver
+import org.xml.sax.ErrorHandler
+import org.xml.sax.InputSource
+import org.xml.sax.SAXNotRecognizedException
+import org.xml.sax.SAXNotSupportedException
+import org.xml.sax.SAXParseException
+
+class DaffodilParseXMLReader(dp: DataProcessor) extends DFDL.DaffodilParseXMLReader {
+  private var contentHandler: ContentHandler = _
+  private var errorHandler: ErrorHandler = _
+  private var dtdHandler: DTDHandler = _
+  private var entityResolver: EntityResolver = _
+  var saxParseResultPropertyValue: ParseResult = _
+  var saxBlobDirectoryPropertyValue: Path = Paths.get(System.getProperty("java.io.tmpdir"))
+  var saxBlobPrefixPropertyValue: String = "daffodil-sax-"
+  var saxBlobSuffixPropertyValue: String = ".blob"
+
+  private val featureMap = mutable.Map[String, Boolean](
+    XMLUtils.SAX_NAMESPACES_FEATURE -> false,
+    XMLUtils.SAX_NAMESPACE_PREFIXES_FEATURE -> false
+  )
+
+  override def getFeature(name: String): Boolean = {
+    if (name == XMLUtils.SAX_NAMESPACES_FEATURE ||
+      name == XMLUtils.SAX_NAMESPACE_PREFIXES_FEATURE) {
+      featureMap(name)
+    } else {
+      throw new SAXNotRecognizedException("Feature unsupported: " + name + ".\n" +
+        "Supported features are: " + featureMap.keys.mkString(", ")
+      )
+    }
+  }
+
+  override def setFeature(name: String, value: Boolean): Unit = {
+    if (name == XMLUtils.SAX_NAMESPACES_FEATURE ||
+      name == XMLUtils.SAX_NAMESPACE_PREFIXES_FEATURE) {
+      featureMap(name) = value
+    } else {
+      throw new SAXNotRecognizedException("Feature unsupported: " + name + ".\n" +
+        "Supported features are: " + featureMap.keys.mkString(", ")
+      )
+    }
+  }
+
+  override def getProperty(name: String): AnyRef = {
+    val prop = name match {
+      case XMLUtils.DAFFODIL_SAX_URN_PARSERESULT => saxParseResultPropertyValue
+      case XMLUtils.DAFFODIL_SAX_URN_BLOBDIRECTORY => saxBlobDirectoryPropertyValue
+      case XMLUtils.DAFFODIL_SAX_URN_BLOBPREFIX => saxBlobPrefixPropertyValue
+      case XMLUtils.DAFFODIL_SAX_URN_BLOBSUFFIX => saxBlobSuffixPropertyValue
+      case _ =>
+        throw new SAXNotRecognizedException("Property unsupported: " + name + ".")
+    }
+    prop
+  }
+
+  override def setProperty(name: String, value: AnyRef): Unit = {
+    try {
+      name match {
+        case XMLUtils.DAFFODIL_SAX_URN_BLOBDIRECTORY => saxBlobDirectoryPropertyValue =
+          value.asInstanceOf[Path]
+        case XMLUtils.DAFFODIL_SAX_URN_BLOBPREFIX => saxBlobPrefixPropertyValue = value
+          .asInstanceOf[String]
+        case XMLUtils.DAFFODIL_SAX_URN_BLOBSUFFIX => saxBlobSuffixPropertyValue = value
+          .asInstanceOf[String]
+        case _ =>
+          throw new SAXNotRecognizedException("Property unsupported: " + name + ".")
+      }
+    } catch {
+      case _: ClassCastException =>
+        throw new SAXNotSupportedException("Unsupported value for property: " + name + ".")
+    }
+  }
+
+  override def setEntityResolver(resolver: EntityResolver): Unit = {
+    entityResolver = resolver
+  }
+
+  override def getEntityResolver: EntityResolver = entityResolver
+
+  override def setDTDHandler(handler: DTDHandler): Unit = {
+    dtdHandler = handler
+  }
+
+  override def getDTDHandler: DTDHandler = dtdHandler
+
+  override def setContentHandler(handler: ContentHandler): Unit = {
+    contentHandler = handler;
+  }
+
+  override def getContentHandler: ContentHandler = contentHandler
+
+  override def setErrorHandler(handler: ErrorHandler): Unit = {
+    errorHandler = handler;
+  }
+
+  override def getErrorHandler: ErrorHandler = errorHandler
+
+  override def parse(input: InputSource): Unit = {
+    val is = input.getByteStream
+    if (is != null) {
+      val isdis = InputSourceDataInputStream(is)
+      parse(isdis)
+    } else {
+      throw new IOException("InputSource must be backed by InputStream")
+    }
+  }
+
+  override def parse(systemId: String): Unit = {
+    throw new IOException("SAX parsing of systemId is unsupported")
+  }
+
+  def parse(isdis: InputSourceDataInputStream): Unit = {
+    val sio = createSAXInfosetOutputter(this)
+    val pr = dp.parse(isdis, sio)
+    saxParseResultPropertyValue = pr.asInstanceOf[ParseResult]
+    handleDiagnostics(pr)
+  }
+
+  def parse(stream: InputStream): Unit = {
+    val isdis = InputSourceDataInputStream(stream)
+    parse(isdis)
+  }
+
+  def parse(arr: Array[Byte]): Unit = {
+    val isdis = InputSourceDataInputStream(arr)
+    parse(isdis)
+  }
+
+  private def handleDiagnostics(pr: DFDL.ParseResult): Unit = {
+    val diagnostics = pr.getDiagnostics
+    val eh = this.getErrorHandler
+    if (diagnostics.nonEmpty && eh != null) {
+      diagnostics.foreach { d =>
+        val spe = {
+          val msg = d.getMessage()
+          val (lineNo, colNo, systemId) = d.getLocationsInSchemaFiles.headOption.map { s =>
+            val sl = s.asInstanceOf[SchemaFileLocation]
+            val ln = sl.lineNumber.getOrElse("0").toInt
+            val cn = sl.columnNumber.getOrElse("0").toInt
+            val sId = sl.uriString
+            (ln, cn, sId)
+          }.getOrElse((0, 0, null))
+
+          val spe = new SAXParseException(msg, null, systemId, lineNo, colNo, d)
+          spe
+        }
+
+        if (d.isError) {
+          eh.error(spe)
+        } else {
+          eh.warning(spe)
+        }
+      }
+    }
+  }
+
+  /**
+   * Creates SAXInfosetOutputter object and attempts to setBlobAttributes on it if
+   * it has at least the blobDirectory property set
+   *
+   * @return SAXInfosetOutputter object with or without blob Attributes set
+   */
+  private def createSAXInfosetOutputter(xmlReader: DaffodilParseXMLReader): SAXInfosetOutputter = {
+    val sioo = new SAXInfosetOutputter(xmlReader)
+    val siof = try {
+      sioo.setBlobAttributes(saxBlobDirectoryPropertyValue, saxBlobPrefixPropertyValue,
+        saxBlobSuffixPropertyValue
+      )
+      sioo
+    } catch {
+      case e: SAXNotSupportedException => sioo
+    }
+    siof
+  }
+}
diff --git a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DataProcessor.scala b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DataProcessor.scala
index 8661844..a8ea86a 100644
--- a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DataProcessor.scala
+++ b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DataProcessor.scala
@@ -19,26 +19,22 @@ package org.apache.daffodil.processors
 
 import java.io.File
 import java.io.IOException
-import java.io.InputStream
 import java.io.ObjectOutputStream
 import java.nio.CharBuffer
 import java.nio.LongBuffer
 import java.nio.channels.Channels
 import java.nio.file.Files
-import java.nio.file.Path
-import java.nio.file.Paths
 import java.util.zip.GZIPOutputStream
 
 import scala.collection.immutable.Queue
-import scala.collection.mutable
-import org.apache.daffodil.Implicits._
+
+import org.apache.daffodil.Implicits._; object INoWarn4 {
+  ImplicitsSuppressUnusedImportWarning() }
 import org.apache.daffodil.api.ValidationException
 import org.apache.daffodil.api.ValidationFailure
 import org.apache.daffodil.api.ValidationResult
 import org.apache.daffodil.api.Validator
 import org.apache.daffodil.validation.XercesValidatorFactory
-; object INoWarn4 {
-  ImplicitsSuppressUnusedImportWarning() }
 import org.apache.daffodil.api.DFDL
 import org.apache.daffodil.api.DaffodilTunables
 import org.apache.daffodil.api.ValidationMode
@@ -50,7 +46,6 @@ import org.apache.daffodil.equality._; object EqualityNoWarn3 {
   EqualitySuppressUnusedImportWarning() }
 import org.apache.daffodil.events.MultipleEventHandler
 import org.apache.daffodil.exceptions.Assert
-import org.apache.daffodil.exceptions.SchemaFileLocation
 import org.apache.daffodil.exceptions.UnsuppressableException
 import org.apache.daffodil.externalvars.Binding
 import org.apache.daffodil.externalvars.ExternalVariablesLoader
@@ -58,7 +53,6 @@ import org.apache.daffodil.infoset.DIElement
 import org.apache.daffodil.infoset.InfosetException
 import org.apache.daffodil.infoset.InfosetInputter
 import org.apache.daffodil.infoset.InfosetOutputter
-import org.apache.daffodil.infoset.SAXInfosetOutputter
 import org.apache.daffodil.infoset.TeeInfosetOutputter
 import org.apache.daffodil.infoset.XMLTextInfosetOutputter
 import org.apache.daffodil.io.BitOrderChangeException
@@ -74,15 +68,6 @@ import org.apache.daffodil.processors.unparsers.UnparseError
 import org.apache.daffodil.util.Maybe
 import org.apache.daffodil.util.Maybe._
 import org.apache.daffodil.util.Misc
-import org.apache.daffodil.xml.XMLUtils
-import org.xml.sax.ContentHandler
-import org.xml.sax.DTDHandler
-import org.xml.sax.EntityResolver
-import org.xml.sax.ErrorHandler
-import org.xml.sax.InputSource
-import org.xml.sax.SAXNotRecognizedException
-import org.xml.sax.SAXNotSupportedException
-import org.xml.sax.SAXParseException
 
 
 /**
@@ -757,168 +742,4 @@ class UnparseResult(dp: DataProcessor, ustate: UState)
   }
 }
 
-class DaffodilParseXMLReader(dp: DataProcessor) extends DFDL.DaffodilParseXMLReader {
-  private var contentHandler: ContentHandler = _
-  private var errorHandler: ErrorHandler = _
-  private var dtdHandler: DTDHandler = _
-  private var entityResolver: EntityResolver = _
-  var saxParseResultPropertyValue: ParseResult = _
-  var saxBlobDirectoryPropertyValue: Path = Paths.get(System.getProperty("java.io.tmpdir"))
-  var saxBlobPrefixPropertyValue: String = "daffodil-sax-"
-  var saxBlobSuffixPropertyValue: String = ".blob"
-
-  private val featureMap = mutable.Map[String, Boolean](
-    XMLUtils.SAX_NAMESPACES_FEATURE -> false,
-    XMLUtils.SAX_NAMESPACE_PREFIXES_FEATURE -> false)
-
-  override def getFeature(name: String): Boolean = {
-    if (name == XMLUtils.SAX_NAMESPACES_FEATURE ||
-      name == XMLUtils.SAX_NAMESPACE_PREFIXES_FEATURE) {
-      featureMap(name)
-    } else {
-      throw new SAXNotRecognizedException("Feature unsupported: " + name + ".\n" +
-        "Supported features are: " + featureMap.keys.mkString(", "))
-      false
-    }
-  }
-
-  override def setFeature(name: String, value: Boolean): Unit = {
-    if (name == XMLUtils.SAX_NAMESPACES_FEATURE ||
-      name == XMLUtils.SAX_NAMESPACE_PREFIXES_FEATURE) {
-      featureMap(name) = value
-    } else {
-      throw new SAXNotRecognizedException("Feature unsupported: " + name + ".\n" +
-        "Supported features are: " + featureMap.keys.mkString(", "))
-    }
-  }
-
-  override def getProperty(name: String): AnyRef = {
-    val prop = name match {
-      case XMLUtils.DAFFODIL_SAX_URN_PARSERESULT => saxParseResultPropertyValue
-      case XMLUtils.DAFFODIL_SAX_URN_BLOBDIRECTORY => saxBlobDirectoryPropertyValue
-      case XMLUtils.DAFFODIL_SAX_URN_BLOBPREFIX => saxBlobPrefixPropertyValue
-      case XMLUtils.DAFFODIL_SAX_URN_BLOBSUFFIX => saxBlobSuffixPropertyValue
-      case _ =>
-        throw new SAXNotRecognizedException("Property unsupported: " + name + ".")
-    }
-    prop
-  }
-
-  override def setProperty(name: String, value: AnyRef): Unit = {
-    try {
-      name match {
-        case XMLUtils.DAFFODIL_SAX_URN_BLOBDIRECTORY => saxBlobDirectoryPropertyValue =
-          value.asInstanceOf[Path]
-        case XMLUtils.DAFFODIL_SAX_URN_BLOBPREFIX => saxBlobPrefixPropertyValue = value
-          .asInstanceOf[String]
-        case XMLUtils.DAFFODIL_SAX_URN_BLOBSUFFIX => saxBlobSuffixPropertyValue = value
-          .asInstanceOf[String]
-        case _ =>
-          throw new SAXNotRecognizedException("Property unsupported: " + name + ".")
-      }
-    } catch {
-      case _: ClassCastException =>
-        throw new SAXNotSupportedException("Unsupported value for property: " + name + "." )
-    }
-  }
-
-  override def setEntityResolver(resolver: EntityResolver): Unit = {
-    entityResolver = resolver
-  }
-
-  override def getEntityResolver: EntityResolver = entityResolver
-
-  override def setDTDHandler(handler: DTDHandler): Unit = {
-    dtdHandler = handler
-  }
-
-  override def getDTDHandler: DTDHandler = dtdHandler
-
-  override def setContentHandler(handler: ContentHandler): Unit = {
-    contentHandler = handler;
-  }
-
-  override def getContentHandler: ContentHandler = contentHandler
 
-  override def setErrorHandler(handler: ErrorHandler): Unit = {
-    errorHandler = handler;
-  }
-
-  override def getErrorHandler: ErrorHandler = errorHandler
-
-  override def parse(input: InputSource): Unit = {
-    val is = input.getByteStream
-    if (is != null) {
-      val isdis = InputSourceDataInputStream(is)
-      parse(isdis)
-    } else {
-      throw new IOException("InputSource must be backed by InputStream")
-    }
-  }
-
-  override def parse(systemId: String): Unit = {
-    throw new IOException("SAX parsing of systemId is unsupported")
-  }
-
-  def parse(isdis: InputSourceDataInputStream): Unit = {
-    val sio = createSAXInfosetOutputter(this)
-    val pr = dp.parse(isdis, sio)
-    saxParseResultPropertyValue = pr.asInstanceOf[ParseResult]
-    handleDiagnostics(pr)
-  }
-
-  def parse(stream: InputStream): Unit = {
-    val isdis = InputSourceDataInputStream(stream)
-    parse(isdis)
-  }
-
-  def parse(arr: Array[Byte]): Unit = {
-    val isdis = InputSourceDataInputStream(arr)
-    parse(isdis)
-  }
-
-  private def handleDiagnostics(pr: DFDL.ParseResult): Unit = {
-    val diagnostics = pr.getDiagnostics
-    val eh = this.getErrorHandler
-    if (diagnostics.nonEmpty && eh != null) {
-      diagnostics.foreach { d =>
-        val spe = {
-          val msg = d.getMessage()
-          val (lineNo, colNo, systemId) = d.getLocationsInSchemaFiles.headOption.map { s =>
-            val sl = s.asInstanceOf[SchemaFileLocation]
-            val ln = sl.lineNumber.getOrElse("0").toInt
-            val cn = sl.columnNumber.getOrElse("0").toInt
-            val sId = sl.uriString
-            (ln, cn, sId)
-          }.getOrElse((0,0, null))
-
-          val spe = new SAXParseException(msg, null, systemId, lineNo, colNo, d)
-          spe
-        }
-
-        if (d.isError) {
-          eh.error(spe)
-        } else {
-          eh.warning(spe)
-        }
-      }
-    }
-  }
-
-  /**
-   * Creates SAXInfosetOutputter object and attempts to setBlobAttributes on it if
-   * it has at least the blobDirectory property set
-   *
-   * @return SAXInfosetOutputter object with or without blob Attributes set
-   */
-  private def createSAXInfosetOutputter(xmlReader: DaffodilParseXMLReader): SAXInfosetOutputter = {
-    val sioo = new SAXInfosetOutputter(xmlReader)
-    val siof = try {
-      sioo.setBlobAttributes(saxBlobDirectoryPropertyValue, saxBlobPrefixPropertyValue, saxBlobSuffixPropertyValue)
-      sioo
-    } catch {
-      case e: SAXNotSupportedException => sioo
-    }
-    siof
-  }
-}
diff --git a/daffodil-tdml-processor/src/main/scala/org/apache/daffodil/tdml/processor/DaffodilTDMLDFDLProcessor.scala b/daffodil-tdml-processor/src/main/scala/org/apache/daffodil/tdml/processor/DaffodilTDMLDFDLProcessor.scala
index 63353f3..523b8c0 100644
--- a/daffodil-tdml-processor/src/main/scala/org/apache/daffodil/tdml/processor/DaffodilTDMLDFDLProcessor.scala
+++ b/daffodil-tdml-processor/src/main/scala/org/apache/daffodil/tdml/processor/DaffodilTDMLDFDLProcessor.scala
@@ -38,9 +38,9 @@ import org.apache.daffodil.debugger.TraceDebuggerRunner
 import org.apache.daffodil.dsom.ExpressionCompilers
 import org.apache.daffodil.exceptions.Assert
 import org.apache.daffodil.externalvars.Binding
-import org.apache.daffodil.infoset.DaffodilParseOutputStreamContentHandler
 import org.apache.daffodil.infoset.ScalaXMLInfosetInputter
 import org.apache.daffodil.io.InputSourceDataInputStream
+import org.apache.daffodil.processors.DaffodilParseOutputStreamContentHandler
 import org.apache.daffodil.processors.DataProcessor
 import org.apache.daffodil.processors.UnparseResult
 import org.apache.daffodil.processors.unparsers.UState