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

[incubator-daffodil] branch master updated: Enable annotation of an XSD with both DFDL annotations and an unrelated annotation language.

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

mbeckerle 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 6ad629d  Enable annotation of an XSD with both DFDL annotations and an unrelated annotation language.
6ad629d is described below

commit 6ad629d19b9c53db117155ed4f99ec565a07e274
Author: Michael Beckerle <mb...@tresys.com>
AuthorDate: Tue Feb 20 10:58:41 2018 -0500

    Enable annotation of an XSD with both DFDL
    annotations and an unrelated annotation language.
    
    Problem is that like DFDL, this additional annotation language defines
    annotation elements that have xsd:attribute definitions in them.
    
    And Daffodil issues SDE on those attribute definitions because DFDL
    doesn't use attributes.
    
    But they're not being used to describe data. They're just being used in
    annotation elements that Daffodil (or any DFDL processor) can simply
    ignore.
    
    To allow import of a full-function XML Schema for an annotation
    language, Daffodil looks at schemas for use of the DFDL_NAMESPACE. If
    the root element of the schema uses the DFDL_NAMESPACE (e.g., to define
    the dfdl prefix), then the file is treated as a DFDL schema. If the
    DFDL_NAMESPACE is not used, then the file is not imported. It is
    ignored.
    
    Rationalized validation. Schemas no longer validated by SchemaSet, but
    centralized into DFDLSchemaFile class.
    
    This changes validation behavior somewhat. Some errors are superceded by
    others now, so negative tests need to change somewhat.
    
    Some tests used to work only because of bugs where validation was not
    occurring but should have been, or passed because validation was
    occurring when it should not have. So more precise use of test runners,
    with fewer tests using modes like validateTDMLFile=false or
    validateDFDLSchemas=false.
    
    DAFFODIL-1907,  DAFFODIL-1473
---
 .../org/apache/daffodil/dsom/DFDLSchemaFile.scala  | 54 +++++++++++++++-------
 .../apache/daffodil/dsom/ElementDeclMixin.scala    |  4 +-
 .../org/apache/daffodil/dsom/SchemaDocument.scala  | 40 ++++++++++++++++
 .../scala/org/apache/daffodil/dsom/SchemaSet.scala | 26 -----------
 .../dsom/SchemaSetIncludesAndImportsMixins.scala   | 11 ++++-
 .../apache/daffodil/dsom/TestDsomCompiler.scala    |  2 +-
 .../scala/org/apache/daffodil/api/WarnID.scala     |  1 +
 .../apache/daffodil/xml/DaffodilXMLLoader.scala    |  4 +-
 .../tdml-with-embedded-schema-errors.tdml          |  3 +-
 .../scala/org/apache/daffodil/TresysTests.scala    | 23 +++------
 ...egrity.dfdl.xsd => otherAnnotationLanguage.xsd} | 24 ++++++++--
 .../section00/general/ref_integrity.dfdl.xsd       |  7 ++-
 .../general/schemaWithOtherAnnotations.dfdl.xsd    | 50 ++++++++++++++++++++
 .../general/testImportOtherAnnotationSchema.tdml}  | 36 ++++++++-------
 .../namespaces/namespaceSpecialChar2.dfdl.xsd      | 10 ++--
 .../dfdl_expressions/expressions_broken.dfdl.xsd   |  2 +-
 .../dfdl_expressions/expressions_broken_2.dfdl.xsd |  2 +-
 .../daffodil/usertests/DefaultProperties.dfdl.xsd  |  4 +-
 .../general/TestImportOtherAnnotationSchema.scala} | 25 ++++------
 .../processing_errors/ProcessingErrors.scala       |  9 ++--
 .../ProcessingErrorsUnparse.scala                  |  3 +-
 .../dfdl_xsdl_subset/TestDFDLSubset.scala          |  8 ++--
 .../daffodil/section05/facets/TestFacets.scala     | 28 ++++++-----
 .../section06/namespaces/TestNamespaces.scala      | 16 ++++---
 .../property_syntax/TestPropertySyntax.scala       |  3 +-
 .../dfdl_expressions/TestDFDLExpressions.scala     | 26 ++++++-----
 26 files changed, 266 insertions(+), 155 deletions(-)

diff --git a/daffodil-core/src/main/scala/org/apache/daffodil/dsom/DFDLSchemaFile.scala b/daffodil-core/src/main/scala/org/apache/daffodil/dsom/DFDLSchemaFile.scala
index c12a6c4..d653794 100644
--- a/daffodil-core/src/main/scala/org/apache/daffodil/dsom/DFDLSchemaFile.scala
+++ b/daffodil-core/src/main/scala/org/apache/daffodil/dsom/DFDLSchemaFile.scala
@@ -28,6 +28,7 @@ import org.apache.daffodil.oolag.OOLAG
 import org.xml.sax.SAXException
 import org.apache.daffodil.util.LogLevel
 import org.apache.daffodil.util.Misc
+import org.apache.daffodil.exceptions.Assert
 
 /**
  * represents one schema document file
@@ -101,18 +102,11 @@ final class DFDLSchemaFile(val sset: SchemaSet,
     validationDiagnostics_ :+= sde
   }
 
-  def fatalError(exception: SAXParseException) = {
-    val sde = new SchemaDefinitionError(this.schemaFileLocation, "Fatal error loading schema due to %s", exception)
-    validationDiagnostics_ :+= sde
-    // parser throws out of fatalErrors.
-  }
-
-  private lazy val loader = {
-    val ldr = new DaffodilXMLLoader(this)
-    // val shouldValidate = sset.validateDFDLSchemas
-    // ldr.setValidation(shouldValidate) // TODO: Validation not occurring JIRA DFDL-1473. Fix later.
-    ldr
-  }
+  /**
+   * Called on a fatal exception. The parser/validator throws the exception after
+   * this call returns.
+   */
+  def fatalError(exception: SAXParseException) = error(exception) // same as non-fatal exception.
 
   private def loadedNode = LV('loadedNode) {
     def die(e: Throwable) = {
@@ -120,7 +114,14 @@ final class DFDLSchemaFile(val sset: SchemaSet,
     }
     val node = try {
       log(LogLevel.Resolver, "Loading %s.", diagnosticDebugName)
-      val node = loader.load(schemaSource)
+      val ldr = new DaffodilXMLLoader(this)
+      //
+      // We do not want to validate here ever, because we have to examine the
+      // root xs:schema eleemnt of a schema to decide if it is a  DFDL schema
+      // at all that we're even supposed to compile.
+      //
+      ldr.setValidation(false)
+      val node = ldr.load(schemaSource)
       schemaDefinitionUnless(node != null, "No XML Node could be loaded from %s.", schemaSource)
       node
     } catch {
@@ -132,8 +133,29 @@ final class DFDLSchemaFile(val sset: SchemaSet,
 
   lazy val node = loadedNode
 
-  def iiXMLSchemaDocument = LV('iiXMLSchemaDocument) {
+  lazy val isDFDLSchemaFile = iiXMLSchemaDocument.isDFDLSchema
+
+  lazy val iiXMLSchemaDocument = LV('iiXMLSchemaDocument) {
     val res = loadXMLSchemaDocument(seenBefore, Some(this))
+    if (res.isDFDLSchema && sset.validateDFDLSchemas) {
+      //
+      // We validate DFDL schemas, only if validation is requested.
+      // Some things, tests generally, want to turn this validation off.
+      //
+
+      val ldr = new DaffodilXMLLoader(this)
+      ldr.setValidation(true)
+      try {
+        ldr.load(schemaSource) // validate as XML file with XML Schema for DFDL Schemas
+        ldr.validateSchema(schemaSource) // validate as XSD (catches UPA errors for example)
+      } catch {
+        case _: org.xml.sax.SAXParseException =>
+          // ok to absorb this. We have captured fatal exceptions in the
+          // error handler. 
+        case e: Exception =>
+          Assert.invariantFailed("Unexpected exception type " + e)
+      }
+    }
     res
   }.value
 
@@ -142,11 +164,9 @@ final class DFDLSchemaFile(val sset: SchemaSet,
     res
   }.value
 
-  private def loadXMLSchemaDocument(before: IIMap, sf: Option[DFDLSchemaFile]) = {
+  private def loadXMLSchemaDocument(before: IIMap, sf: Option[DFDLSchemaFile]): XMLSchemaDocument = {
     val sd = node match {
       case <schema>{ _* }</schema> if (NS(node.namespace) == XMLUtils.xsdURI) => {
-        // top level is a schema.
-
         val sd = new XMLSchemaDocument(node, sset, Some(iiParent), sf, before, false)
         sd
       }
diff --git a/daffodil-core/src/main/scala/org/apache/daffodil/dsom/ElementDeclMixin.scala b/daffodil-core/src/main/scala/org/apache/daffodil/dsom/ElementDeclMixin.scala
index 320178e..d5b0dda 100644
--- a/daffodil-core/src/main/scala/org/apache/daffodil/dsom/ElementDeclMixin.scala
+++ b/daffodil-core/src/main/scala/org/apache/daffodil/dsom/ElementDeclMixin.scala
@@ -99,8 +99,8 @@ trait ElementDeclMixin
             // Note: Validation of the DFDL Schema doesn't necessarily check referential integrity
             // or other complex constraints like conflicting names.
             // So we check it here explicitly.
-            case (None, None) => schemaDefinitionError("No type definition found for '%s'.", namedTypeQName)
-            case (Some(_), Some(_)) => schemaDefinitionError("Both a simple and a complex type definition found for '%s'", namedTypeQName)
+            case (None, None) => schemaDefinitionError("No type definition found for '%s'.", qn.toPrettyString)
+            case (Some(_), Some(_)) => schemaDefinitionError("Both a simple and a complex type definition found for '%s'", qn.toPrettyString)
           }
           res
         }
diff --git a/daffodil-core/src/main/scala/org/apache/daffodil/dsom/SchemaDocument.scala b/daffodil-core/src/main/scala/org/apache/daffodil/dsom/SchemaDocument.scala
index 534cc1d..4a87c21 100644
--- a/daffodil-core/src/main/scala/org/apache/daffodil/dsom/SchemaDocument.scala
+++ b/daffodil-core/src/main/scala/org/apache/daffodil/dsom/SchemaDocument.scala
@@ -24,6 +24,7 @@ import org.apache.daffodil.exceptions.Assert
 import org.apache.daffodil.schema.annotation.props.SeparatorSuppressionPolicyMixin
 import org.apache.daffodil.schema.annotation.props.gen.Format_AnnotationMixin
 import org.apache.daffodil.api.WarnID
+import org.apache.daffodil.xml.XMLUtils
 
 /**
  * A schema document corresponds to one file usually named with an ".xsd" extension.
@@ -125,6 +126,45 @@ final class XMLSchemaDocument(xmlArg: Node,
     res
   }.value
 
+  /**
+   *  True if root xs:schema element has an xmlns that uses the DFDL URI.
+   *
+   *  This could be a prefix definition (Most likely xmlns:dfdl='...' but could
+   *  be some other prefix.)
+   *
+   *  Or very very unlikely, it could be the default namespace.
+   */
+  private def hasDFDLNamespaceDefinition: Boolean = {
+    val scope = xml.scope
+    val pre = scope.getPrefix(XMLUtils.DFDL_NAMESPACE)
+    val hasSomePrefixForDFDLNamespace = pre ne null
+    lazy val hasDefaultNamespaceAsDFDLNamespace = {
+      val defaultNS = scope.getURI(null)
+      defaultNS == XMLUtils.DFDL_NAMESPACE.toString
+    }
+    val res = hasSomePrefixForDFDLNamespace || hasDefaultNamespaceAsDFDLNamespace
+    res
+  }
+
+  /**
+   * True if this is a DFDL schema that Daffodil should process.
+   * False if this schema should be ignored because it has no DFDL annotations.
+   *
+   * We will ignore this import/include if it does not use the DFDL namespace
+   * definition for a prefix (or the default) on the xs:schema element.
+   *
+   * We do this so that other annotation languages can co-exist with DFDL.
+   * That is, they can use all of XML Schema including things DFDL doesn't allow
+   * like attribute decls, but only when those schemas are only needed
+   * to process non-DFDL annotations. Since Daffodil ignores non-DFDL annotations
+   * entirely, Daffodil won't run into these non-DFDL allowed things like
+   * attribute decls. But validators like Xerces will see the regular
+   * import/include and process normally, which enables validation of
+   * all annotations, DFDL and otherwise.
+   *
+   * Further discussion - see the comments on JIRA ticket DAFFODIL-1909
+   */
+  lazy val isDFDLSchema = hasDFDLNamespaceDefinition
 }
 
 /**
diff --git a/daffodil-core/src/main/scala/org/apache/daffodil/dsom/SchemaSet.scala b/daffodil-core/src/main/scala/org/apache/daffodil/dsom/SchemaSet.scala
index 5b920b1..017ae68 100644
--- a/daffodil-core/src/main/scala/org/apache/daffodil/dsom/SchemaSet.scala
+++ b/daffodil-core/src/main/scala/org/apache/daffodil/dsom/SchemaSet.scala
@@ -34,7 +34,6 @@ import java.io.File
 import org.apache.daffodil.xml.DFDLCatalogResolver
 import org.apache.daffodil.api.DaffodilSchemaSource
 import org.apache.daffodil.api.UnitTestSchemaSource
-import org.apache.daffodil.util.Misc
 import org.apache.daffodil.schema.annotation.props.LookupLocation
 import org.apache.daffodil.api.DaffodilTunables
 
@@ -78,7 +77,6 @@ final class SchemaSet(
     requiredEvaluations(checkForDuplicateTopLevels())
     requiredEvaluations(this.allTopLevels)
   }
-  requiredEvaluations(validateSchemaFiles)
   requiredEvaluations(variableMap)
 
   lazy val resolver = DFDLCatalogResolver.get
@@ -100,30 +98,6 @@ final class SchemaSet(
    */
   override lazy val uriString: String = schemaSources(0).uriForLoading.toString
 
-  /**
-   * We need to use the loader here to validate the DFDL Schema.
-   */
-  private lazy val loader = new DaffodilXMLLoader(new ValidateSchemasErrorHandler(this))
-
-  /**
-   * Validates the DFDL Schema files present in the schemaFilesArg.
-   * Compiles a list of all errors and warnings before issuing them.
-   *
-   * Issues SchemaDefinitionWarnings for DFDLSchemaValidationWarnings.
-   * Issues SchemaDefinitionErrors for DFDLSchemaValidationErrors.
-   */
-  private def validateSchemaFiles = LV('validateSchemaFiles) {
-    // TODO: DFDL-400 remove this flag check once we've fixed all affected tests.
-    if (validateDFDLSchemas) {
-      schemaSources.foreach(f =>
-        try {
-          loader.validateSchema(f)
-        } catch {
-          case e: DFDLSchemaValidationException => SDE(Misc.getSomeMessage(e).get)
-        })
-    }
-  }.value
-
   lazy val checkAllTopLevel = checkAllTopLevelArg
 
   override def warn(th: Diagnostic) = oolagWarn(th)
diff --git a/daffodil-core/src/main/scala/org/apache/daffodil/dsom/SchemaSetIncludesAndImportsMixins.scala b/daffodil-core/src/main/scala/org/apache/daffodil/dsom/SchemaSetIncludesAndImportsMixins.scala
index 74233b0..f42c2bc 100644
--- a/daffodil-core/src/main/scala/org/apache/daffodil/dsom/SchemaSetIncludesAndImportsMixins.scala
+++ b/daffodil-core/src/main/scala/org/apache/daffodil/dsom/SchemaSetIncludesAndImportsMixins.scala
@@ -22,6 +22,7 @@ import org.apache.daffodil.util._
 import IIUtils._
 import org.apache.daffodil.xml.XMLUtils
 import org.apache.daffodil.util.Delay
+import org.apache.daffodil.api.WarnID
 
 /**
  * Mixin for SchemaSet
@@ -63,7 +64,15 @@ trait SchemaSetIncludesAndImportsMixin { self: SchemaSet =>
     val sfl = sa.value.flatMap {
       case (_, ii) => {
         val sf = ii.iiSchemaFileMaybe // maybe not if we've already seen this file for the same namespace.
-        sf
+        sf.filter { f =>
+          if (f.isDFDLSchemaFile)
+            true
+          else {
+            f.SDW(WarnID.IgnoreImport, "Non-DFDL Schema file ignored. Does not have DFDL namespace definition on schema root element.\n" +
+              "Add xmlns:dfdl='%s' to the root element if this file must be part of the DFDL schema.", XMLUtils.DFDL_NAMESPACE)
+            false
+          }
+        }
       }
     }.toList
     sfl
diff --git a/daffodil-core/src/test/scala/org/apache/daffodil/dsom/TestDsomCompiler.scala b/daffodil-core/src/test/scala/org/apache/daffodil/dsom/TestDsomCompiler.scala
index cdb6769..9e5798f 100644
--- a/daffodil-core/src/test/scala/org/apache/daffodil/dsom/TestDsomCompiler.scala
+++ b/daffodil-core/src/test/scala/org/apache/daffodil/dsom/TestDsomCompiler.scala
@@ -110,7 +110,7 @@ class TestDsomCompiler extends Logging {
 
   // FIXME - convert this test to TDML or drop if there is coverage other places.
   @Test def testTypeReferentialError2() {
-    val sch: Node = <schema xmlns="http://www.w3.org/2001/XMLSchema" targetNamespace="http://example.com">
+    val sch: Node = <schema xmlns="http://www.w3.org/2001/XMLSchema" targetNamespace="http://example.com" xmlns:dfdl="http://www.ogf.org/dfdl/dfdl-1.0/">
                       <element name="foo" type="bar"/><!-- Illegal: no prefix on name of the type. -->
                       <complexType name="bar">
                         <sequence/>
diff --git a/daffodil-lib/src/main/scala/org/apache/daffodil/api/WarnID.scala b/daffodil-lib/src/main/scala/org/apache/daffodil/api/WarnID.scala
index 5d5178d..b7cafea 100644
--- a/daffodil-lib/src/main/scala/org/apache/daffodil/api/WarnID.scala
+++ b/daffodil-lib/src/main/scala/org/apache/daffodil/api/WarnID.scala
@@ -62,6 +62,7 @@ object WarnID extends PropsEnum[WarnID] {
   case object EscapeSchemeRefUndefined extends WarnID; forceConstruction(EscapeSchemeRefUndefined)
   case object FacetExplicitLengthOutOfRange extends WarnID; forceConstruction(FacetExplicitLengthOutOfRange)
   case object InconsistentLengthKind extends WarnID; forceConstruction(InconsistentLengthKind)
+  case object IgnoreImport extends WarnID; forceConstruction(IgnoreImport)
   case object MultipleChoiceBranches extends WarnID; forceConstruction(MultipleChoiceBranches)
   case object NamespaceDifferencesOnly extends WarnID; forceConstruction(NamespaceDifferencesOnly)
   case object NoEmptyDefault extends WarnID; forceConstruction(NoEmptyDefault)
diff --git a/daffodil-lib/src/main/scala/org/apache/daffodil/xml/DaffodilXMLLoader.scala b/daffodil-lib/src/main/scala/org/apache/daffodil/xml/DaffodilXMLLoader.scala
index 09e5731..43c69fa 100644
--- a/daffodil-lib/src/main/scala/org/apache/daffodil/xml/DaffodilXMLLoader.scala
+++ b/daffodil-lib/src/main/scala/org/apache/daffodil/xml/DaffodilXMLLoader.scala
@@ -518,7 +518,9 @@ class DaffodilXMLLoader(val errorHandler: org.xml.sax.ErrorHandler) {
   def load(source: DaffodilSchemaSource): scala.xml.Node = {
     var xercesNode: Node = null
     if (doValidation) {
-      xercesNode = xercesAdapter.load(source.newInputSource()) // validates
+      xercesNode =
+        xercesAdapter.load(source.newInputSource()) // validates
+
       if (xercesNode == null) return null
       // Note: we don't call xercesAdapter.validateSchema(source)
       // here, because this is an XML loader, not necessarily
diff --git a/daffodil-tdml/src/test/resources/tdml-with-embedded-schema-errors.tdml b/daffodil-tdml/src/test/resources/tdml-with-embedded-schema-errors.tdml
index 187743b..52076c8 100644
--- a/daffodil-tdml/src/test/resources/tdml-with-embedded-schema-errors.tdml
+++ b/daffodil-tdml/src/test/resources/tdml-with-embedded-schema-errors.tdml
@@ -32,8 +32,7 @@
       <tdml:documentPart type="text"><![CDATA[abcdef]]></tdml:documentPart>
     </tdml:document>
     <tdml:errors>
-      <tdml:error>xs:format</tdml:error>
-      <tdml:error>invalid dfdl annotation</tdml:error>
+      <tdml:error>'import' tag should be added to</tdml:error>
       <tdml:error>fn:string</tdml:error>
     </tdml:errors>
   </tdml:parserTestCase>
diff --git a/daffodil-test-ibm1/src/test/scala/org/apache/daffodil/TresysTests.scala b/daffodil-test-ibm1/src/test/scala/org/apache/daffodil/TresysTests.scala
index 7308ea3..c63ddb9 100644
--- a/daffodil-test-ibm1/src/test/scala/org/apache/daffodil/TresysTests.scala
+++ b/daffodil-test-ibm1/src/test/scala/org/apache/daffodil/TresysTests.scala
@@ -32,13 +32,10 @@ object TresysTests {
 
   val testDir = "/test-suite/tresys-contributed/"
 
-  val delimited = testDir + "dpaext1.tdml"
-  lazy val runnerDelimited = new DFDLTestSuite(Misc.getRequiredResource(delimited))
+  val runnerDelimited = Runner(testDir, "dpaext1.tdml")
 
-  val td = testDir + "multiple-diagnostics.tdml"
-  lazy val runnerMD = new DFDLTestSuite(Misc.getRequiredResource(td), validateTDMLFile = true, validateDFDLSchemas = false)
-
-  runnerMD.setCheckAllTopLevel(true)
+  val runnerMD = Runner(testDir, "multiple-diagnostics.tdml", compileAllTopLevel = true)
+  val runnerMD_NV = Runner(testDir, "multiple-diagnostics.tdml", compileAllTopLevel = true, validateDFDLSchemas = false)
 
   val ax = testDir + "AX.tdml"
   lazy val runnerAX = new DFDLTestSuite(Misc.getRequiredResource(ax))
@@ -62,11 +59,7 @@ object TresysTests {
   //  val ab9 = testDir + "AB009.tdml"
   //  lazy val runnerAB9 = new DFDLTestSuite(Misc.getRequiredResource(ab9))
 
-  val rd = testDir + "runtime-diagnostics.tdml"
-  lazy val runnerRD = new DFDLTestSuite(Misc.getRequiredResource(rd),
-    validateTDMLFile = false)
-
-  runnerRD.setCheckAllTopLevel(true)
+  lazy val runnerRD = Runner(testDir, "runtime-diagnostics.tdml", compileAllTopLevel = true, validateTDMLFile = false)
 
   val sq = testDir + "sequence.tdml"
   lazy val runnerSQ = new DFDLTestSuite(Misc.getRequiredResource(sq))
@@ -109,13 +102,9 @@ class TresysTests {
 
   @Test def test_AV003() { runnerAV003.runOneTest("AV003") } // needs date
 
-  @Test def test_multiple_diagnostics1() {
-    runnerMD.runOneTest("twoMissingTypeDefErrors")
-  }
+  @Test def test_multiple_diagnostics1() { runnerMD.runOneTest("twoMissingTypeDefErrors") }
   @Test def test_multiple_diagnostics2() { runnerMD.runOneTest("manyErrors1") }
-  @Test def test_multiple_diagnostics3() { // LoggingDefaults.setLoggingLevel(LogLevel.Compile)
-    runnerMD.runOneTest("manyErrors2")
-  }
+  @Test def test_multiple_diagnostics3() { runnerMD_NV.runOneTest("manyErrors2") }
 
   // not found. Debug later.
   // @Test def test_duplicateDefineFormatsOneSchema() { runnerMD.runOneTest("duplicateDefineFormatsOneSchema") }
diff --git a/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/ref_integrity.dfdl.xsd b/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/otherAnnotationLanguage.xsd
similarity index 55%
copy from daffodil-test/src/test/resources/org/apache/daffodil/section00/general/ref_integrity.dfdl.xsd
copy to daffodil-test/src/test/resources/org/apache/daffodil/section00/general/otherAnnotationLanguage.xsd
index 1c051dd..4a7061c 100644
--- a/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/ref_integrity.dfdl.xsd
+++ b/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/otherAnnotationLanguage.xsd
@@ -17,9 +17,23 @@
 -->
 
 <schema xmlns="http://www.w3.org/2001/XMLSchema"
-targetNamespace="http://example.com">
-	<element name="foo" type="bar"/><!-- Illegal: no prefix on name of the type. -->
-	<complexType name="bar">
-		<sequence/>
-	</complexType>
+  targetNamespace="urn:otherAnnotationLanguage" 
+  xmlns:tns="urn:otherAnnotationLanguage"
+  xmlns:xs="http://www.w3.org/2001/XMLSchema"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" 
+  xmlns:fn="http://www.w3.org/2005/xpath-functions">
+
+  <element name="otherAnnotation">
+    <complexType>
+       <attribute ref="tns:otherAnnotationAttribute"/>
+    </complexType>
+  </element>
+
+  <!-- 
+  Some other annotation elements that are not DFDL annotations may also
+  have attributes. These attributes should be legal in DFDL, as they're not describing data, they're
+  just describing annotations on the schema that a DFDL processor should be ignoring.
+   -->
+  <attribute name="otherAnnotationAttribute" type="string"/>
+
 </schema>
diff --git a/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/ref_integrity.dfdl.xsd b/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/ref_integrity.dfdl.xsd
index 1c051dd..6d72cdb 100644
--- a/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/ref_integrity.dfdl.xsd
+++ b/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/ref_integrity.dfdl.xsd
@@ -17,7 +17,12 @@
 -->
 
 <schema xmlns="http://www.w3.org/2001/XMLSchema"
-targetNamespace="http://example.com">
+  targetNamespace="http://example.com"
+  xmlns:dfdl="http://www.ogf.org/dfdl/dfdl-1.0/">
+    <!-- 
+      this schema doesn't need to use dfdl symbols, but to make Daffodil
+      even compile it we have to mention the DFDL namespace URI above.
+     -->
 	<element name="foo" type="bar"/><!-- Illegal: no prefix on name of the type. -->
 	<complexType name="bar">
 		<sequence/>
diff --git a/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/schemaWithOtherAnnotations.dfdl.xsd b/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/schemaWithOtherAnnotations.dfdl.xsd
new file mode 100644
index 0000000..d0d4cfb
--- /dev/null
+++ b/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/schemaWithOtherAnnotations.dfdl.xsd
@@ -0,0 +1,50 @@
+<?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.
+-->
+
+<schema xmlns="http://www.w3.org/2001/XMLSchema"
+  xmlns:oth="urn:otherAnnotationLanguage"
+  xmlns:dfdl="http://www.ogf.org/dfdl/dfdl-1.0/"
+  xmlns:xs="http://www.w3.org/2001/XMLSchema"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" 
+  xmlns:fn="http://www.w3.org/2005/xpath-functions"
+  targetNamespace="urn:foo"
+  xmlns:tns="urn:foo">
+
+  <xs:include schemaLocation="org/apache/daffodil/xsd/DFDLGeneralFormat.dfdl.xsd" />
+
+  <!-- 
+    If you comment out the import below, then the other annotations are ignored.
+    Buf it this is allowed to import, you get an SDE on the attribute decls in it.
+    -->
+  <xs:import namespace="urn:otherAnnotationLanguage" schemaLocation="otherAnnotationLanguage.xsd" />
+
+  <xs:annotation>
+    <xs:appinfo source="http://www.ogf.org/dfdl/">
+      <dfdl:format ref="tns:GeneralFormat" />
+    </xs:appinfo>
+  </xs:annotation>
+
+  <xs:element name="r1" type="xs:string" dfdl:lengthKind="delimited">
+    <xs:annotation>
+      <xs:appinfo source="urn:otherAnnotationLanguage">
+        <oth:otherAnnotation oth:otherAnnotationAttribute="other"/>
+      </xs:appinfo>
+    </xs:annotation>
+  </xs:element>
+  
+</schema>
diff --git a/daffodil-test/src/test/resources/org/apache/daffodil/section23/dfdl_expressions/expressions_broken.dfdl.xsd b/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/testImportOtherAnnotationSchema.tdml
similarity index 52%
copy from daffodil-test/src/test/resources/org/apache/daffodil/section23/dfdl_expressions/expressions_broken.dfdl.xsd
copy to daffodil-test/src/test/resources/org/apache/daffodil/section00/general/testImportOtherAnnotationSchema.tdml
index 60cf938..abbd26c 100644
--- a/daffodil-test/src/test/resources/org/apache/daffodil/section23/dfdl_expressions/expressions_broken.dfdl.xsd
+++ b/daffodil-test/src/test/resources/org/apache/daffodil/section00/general/testImportOtherAnnotationSchema.tdml
@@ -16,21 +16,25 @@
   limitations under the License.
 -->
 
-<xs:schema xmlns:dfdl="http://www.ogf.org/dfdl/dfdl-1.0/" xmlns:xs="http://www.w3.org/2001/XMLSchema"
-           xmlns:ex="http://example.com">
+<tdml:testSuite suiteName="ImportOtherAnnotationSchema"
+  description="Tests for importing other annotation languages to determine if DFDL interacts with them." 
+  xmlns:tdml="http://www.ibm.com/xmlns/dfdl/testData"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" 
+  xmlns:xs="http://www.w3.org/2001/XMLSchema"
+  xmlns:ex="http://example.com" 
+  xmlns:daf="urn:ogf:dfdl:2013:imp:daffodil.apache.org:2018:ext"
+  xmlns:oth="urn:otherAnnotationLanguage">
+  
 
-  <xs:annotation>
-    <xs:appinfo source="http://www.ogf.org/dfdl/">
-      <dfdl:format ref="ex:GeneralFormat" />
-    </xs:appinfo>
-  </xs:annotation>
+  <tdml:parserTestCase name="importOtherAnnotationSchema1" root="r1" model="schemaWithOtherAnnotations.dfdl.xsd">
 
-    <xs:element name="e1" type="xs:int">
-        <xs:annotation> 
-          <xs:appinfo source="http://www.ogf.org/dfdl/" >
-           <dfdl:element representation="hieroglyphs" lengthKind="implicit" />
-          </xs:appinfo>
-        </xs:annotation>
-    </xs:element>
-    
-</xs:schema>
+  <tdml:document>foo</tdml:document>
+  <tdml:infoset>
+    <tdml:dfdlInfoset>
+      <ex:r1>foo</ex:r1>
+    </tdml:dfdlInfoset>
+  </tdml:infoset>
+
+</tdml:parserTestCase>
+
+</tdml:testSuite>
diff --git a/daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/namespaceSpecialChar2.dfdl.xsd b/daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/namespaceSpecialChar2.dfdl.xsd
index ed4be9c..7c1c18a 100644
--- a/daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/namespaceSpecialChar2.dfdl.xsd
+++ b/daffodil-test/src/test/resources/org/apache/daffodil/section06/namespaces/namespaceSpecialChar2.dfdl.xsd
@@ -16,13 +16,15 @@
   limitations under the License.
 -->
 
-<имен:schema xmlns:имен="http://www.w3.org/2001/XMLSchema" xmlns:dfdl="http://www.ogf.org/dfdl/dfdl-1.0/" xmlns:·b="http://example.com" targetNamespace="http://example.com">
+<имен:schema xmlns:имен="http://www.w3.org/2001/XMLSchema" xmlns:dfdl="http://www.ogf.org/dfdl/dfdl-1.0/" xmlns::b="http://example.com" targetNamespace="http://example.com">
+      
       
   <имен:annotation>
     <имен:appinfo source="http://www.ogf.org/dfdl/">
-      <dfdl:format separator="" alignment="1" alignmentUnits="bits" lengthUnits="bytes"
+      <dfdl:format separator="" alignment="1" alignmentUnits="bytes" lengthUnits="bytes"
         trailingSkip="0" initiator="" terminator="" leadingSkip='0' textTrimKind="none" initiatedContent="no"
-        ignoreCase="no" representation="text" textNumberRep="standard" encoding="ASCII"/>
+        ignoreCase="no" representation="text" textNumberRep="standard" encoding="ASCII"
+        textPadKind="none"/>
     </имен:appinfo>
   </имен:annotation>
   
@@ -30,6 +32,6 @@
       <имен:restriction base="имен:int"/>
     </имен:simpleType>
 
-    <имен:element name="one" type="·b:simTyp"/>
+    <имен:element name="one" type=":b:simTyp"/>
   
 </имен:schema>
diff --git a/daffodil-test/src/test/resources/org/apache/daffodil/section23/dfdl_expressions/expressions_broken.dfdl.xsd b/daffodil-test/src/test/resources/org/apache/daffodil/section23/dfdl_expressions/expressions_broken.dfdl.xsd
index 60cf938..3b580e4 100644
--- a/daffodil-test/src/test/resources/org/apache/daffodil/section23/dfdl_expressions/expressions_broken.dfdl.xsd
+++ b/daffodil-test/src/test/resources/org/apache/daffodil/section23/dfdl_expressions/expressions_broken.dfdl.xsd
@@ -21,7 +21,7 @@
 
   <xs:annotation>
     <xs:appinfo source="http://www.ogf.org/dfdl/">
-      <dfdl:format ref="ex:GeneralFormat" />
+      <dfdl:format  />
     </xs:appinfo>
   </xs:annotation>
 
diff --git a/daffodil-test/src/test/resources/org/apache/daffodil/section23/dfdl_expressions/expressions_broken_2.dfdl.xsd b/daffodil-test/src/test/resources/org/apache/daffodil/section23/dfdl_expressions/expressions_broken_2.dfdl.xsd
index 2f435e0..f96bab1 100644
--- a/daffodil-test/src/test/resources/org/apache/daffodil/section23/dfdl_expressions/expressions_broken_2.dfdl.xsd
+++ b/daffodil-test/src/test/resources/org/apache/daffodil/section23/dfdl_expressions/expressions_broken_2.dfdl.xsd
@@ -21,7 +21,7 @@
 
   <xs:annotation>
     <xs:appinfo source="http://www.ogf.org/dfdl/">
-      <dfdl:format ref="ex:GeneralFormat" />
+      <dfdl:format  />
     </xs:appinfo>
   </xs:annotation>
     
diff --git a/daffodil-test/src/test/resources/org/apache/daffodil/usertests/DefaultProperties.dfdl.xsd b/daffodil-test/src/test/resources/org/apache/daffodil/usertests/DefaultProperties.dfdl.xsd
index 5e80d27..15d9e5a 100644
--- a/daffodil-test/src/test/resources/org/apache/daffodil/usertests/DefaultProperties.dfdl.xsd
+++ b/daffodil-test/src/test/resources/org/apache/daffodil/usertests/DefaultProperties.dfdl.xsd
@@ -16,7 +16,7 @@
   limitations under the License.
 -->
 
-<schema xmlns="http://www.w3.org/2001/XMLSchema" xmlns:dfdl="http://www.ogf.org/dfdl/"
+<schema xmlns="http://www.w3.org/2001/XMLSchema" xmlns:dfdl="http://www.ogf.org/dfdl/dfdl-1.0/"
     xmlns:fn="http://www.w3.org/2005/xpath-functions"
 >
 
@@ -54,7 +54,7 @@
 					textNumberPadCharacter="%SP;" textNumberRep="standard"
 					textNumberRounding="pattern" textNumberRoundingMode="roundUp" textPadKind="none"
 					textStandardBase="10" textStandardDecimalSeparator="."
-					textStandardExponentCharacter="E" textStandardGroupingSeparator=","
+					textStandardExponentRep="E" textStandardGroupingSeparator=","
 					textStandardInfinityRep="Inf" textStandardNaNRep="NaN" textStandardZeroRep=""
 					textStringJustification="left" textStringPadCharacter="%SP;" textTrimKind="none"
 					textZonedSignStyle="asciiStandard" trailingSkip="0"
diff --git a/daffodil-test/src/test/scala/org/apache/daffodil/section05/dfdl_xsdl_subset/TestDFDLSubset.scala b/daffodil-test/src/test/scala/org/apache/daffodil/section00/general/TestImportOtherAnnotationSchema.scala
similarity index 52%
copy from daffodil-test/src/test/scala/org/apache/daffodil/section05/dfdl_xsdl_subset/TestDFDLSubset.scala
copy to daffodil-test/src/test/scala/org/apache/daffodil/section00/general/TestImportOtherAnnotationSchema.scala
index be19737..4d8f4f5 100644
--- a/daffodil-test/src/test/scala/org/apache/daffodil/section05/dfdl_xsdl_subset/TestDFDLSubset.scala
+++ b/daffodil-test/src/test/scala/org/apache/daffodil/section00/general/TestImportOtherAnnotationSchema.scala
@@ -15,33 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.daffodil.section05.dfdl_xsdl_subset
+package org.apache.daffodil.section00.general
 
 import org.junit.Test
 import org.apache.daffodil.tdml.Runner
 import org.junit.AfterClass
 
-object TestDFDLSubset {
+object TestImportOtherAnnotationSchema {
+  val testDir = "/org/apache/daffodil/section00/general/"
+  val runner = Runner(testDir, "testImportOtherAnnotationSchema.tdml", validateDFDLSchemas = false)
 
-  val testDir = "/org/apache/daffodil/section05/dfdl_xsdl_subset/"
-  val runner = Runner(testDir, "DFDLSubset.tdml", validateTDMLFile = true, validateDFDLSchemas = false)
-
-  @AfterClass def tearDown() {
+  @AfterClass def shutDown {
     runner.reset
   }
-
 }
 
-class TestDFDLSubset {
+class TestImportOtherAnnotationSchema {
 
-  import TestDFDLSubset._
+  import TestImportOtherAnnotationSchema._
 
-  @Test def test_groupRefGroupRef() { { runner.runOneTest("groupRefGroupRef") } }
-  @Test def test_refInitiator3() { { runner.runOneTest("refInitiator3") } }
-  @Test def test_groupRef() { { runner.runOneTest("groupRef") } }
-  @Test def test_groupRefChoice() { runner.runOneTest("groupRefChoice") }
-  @Test def test_badGroupRef() { { runner.runOneTest("badGroupRef") } }
-  @Test def test_badSeq() { { runner.runOneTest("badSeq") } }
+  //DFDL-1907
+  @Test def test_importOtherAnnotationSchema1() { runner.runOneTest("importOtherAnnotationSchema1") }
 
-  @Test def test_groupRefDFDL() { runner.runOneTest("groupRefDFDL") }
 }
diff --git a/daffodil-test/src/test/scala/org/apache/daffodil/section02/processing_errors/ProcessingErrors.scala b/daffodil-test/src/test/scala/org/apache/daffodil/section02/processing_errors/ProcessingErrors.scala
index 7170fa0..455727c 100644
--- a/daffodil-test/src/test/scala/org/apache/daffodil/section02/processing_errors/ProcessingErrors.scala
+++ b/daffodil-test/src/test/scala/org/apache/daffodil/section02/processing_errors/ProcessingErrors.scala
@@ -18,21 +18,19 @@
 package org.apache.daffodil.section02.processing_errors
 
 import org.junit.Test
-import org.apache.daffodil.util._
-import org.apache.daffodil.tdml.DFDLTestSuite
 import org.apache.daffodil.tdml.Runner
 import org.junit.AfterClass
 
 object TestProcessingErrors {
   val testDir = "/org/apache/daffodil/section02/processing_errors/"
-  val aa = testDir + "dfdl-schema-validation-diagnostics.tdml"
-  var runner = new DFDLTestSuite(Misc.getRequiredResource(aa), validateTDMLFile = false, validateDFDLSchemas = false)
+
+  val runner = Runner(testDir, "dfdl-schema-validation-diagnostics.tdml", validateTDMLFile = false)
 
   val runner02 = Runner(testDir, "ProcessingErrors.tdml", validateTDMLFile = false, validateDFDLSchemas = false)
   val runner02Validate = Runner(testDir, "ProcessingErrors.tdml", validateTDMLFile = true, validateDFDLSchemas = true)
 
   @AfterClass def shutDown {
-    runner = null
+    runner.reset
     runner02.reset
     runner02Validate.reset
   }
@@ -43,7 +41,6 @@ class TestProcessingErrors {
 
   import TestProcessingErrors._
 
-  runner.setCheckAllTopLevel(true) // check every top level construct. Not just the one under specific test.
   @Test def test_twoDFDLSchemaValidationErrors() { runner.runOneTest("twoDFDLSchemaValidationErrors") }
   @Test def test_twoDFDLSchemaValidationErrors2() { runner.runOneTest("twoDFDLSchemaValidationErrors2") }
   @Test def test_fiveDFDLSchemaValidationErrors() { runner.runOneTest("fiveDFDLSchemaValidationErrors") }
diff --git a/daffodil-test/src/test/scala/org/apache/daffodil/section02/processing_errors/ProcessingErrorsUnparse.scala b/daffodil-test/src/test/scala/org/apache/daffodil/section02/processing_errors/ProcessingErrorsUnparse.scala
index d7c78c6..b56b046 100644
--- a/daffodil-test/src/test/scala/org/apache/daffodil/section02/processing_errors/ProcessingErrorsUnparse.scala
+++ b/daffodil-test/src/test/scala/org/apache/daffodil/section02/processing_errors/ProcessingErrorsUnparse.scala
@@ -25,7 +25,8 @@ object TestProcessingErrorsUnparse {
   val testDir = "/org/apache/daffodil/section02/processing_errors/"
 
   val runner02 = Runner(testDir, "ProcessingErrorsUnparse.tdml", validateTDMLFile = false, validateDFDLSchemas = false)
-  val runner02Validate = Runner(testDir, "ProcessingErrorsUnparse.tdml", validateTDMLFile = true, validateDFDLSchemas = true)
+  val runner02Validate = Runner(testDir, "ProcessingErrorsUnparse.tdml", validateTDMLFile = true, validateDFDLSchemas = true,
+    compileAllTopLevel = true)
 
   @AfterClass def shutDown {
     runner02.reset
diff --git a/daffodil-test/src/test/scala/org/apache/daffodil/section05/dfdl_xsdl_subset/TestDFDLSubset.scala b/daffodil-test/src/test/scala/org/apache/daffodil/section05/dfdl_xsdl_subset/TestDFDLSubset.scala
index be19737..9a14b32 100644
--- a/daffodil-test/src/test/scala/org/apache/daffodil/section05/dfdl_xsdl_subset/TestDFDLSubset.scala
+++ b/daffodil-test/src/test/scala/org/apache/daffodil/section05/dfdl_xsdl_subset/TestDFDLSubset.scala
@@ -24,10 +24,12 @@ import org.junit.AfterClass
 object TestDFDLSubset {
 
   val testDir = "/org/apache/daffodil/section05/dfdl_xsdl_subset/"
-  val runner = Runner(testDir, "DFDLSubset.tdml", validateTDMLFile = true, validateDFDLSchemas = false)
+  val runner = Runner(testDir, "DFDLSubset.tdml")
+  val runnerNV = Runner(testDir, "DFDLSubset.tdml", validateDFDLSchemas=false)
 
   @AfterClass def tearDown() {
     runner.reset
+    runnerNV.reset
   }
 
 }
@@ -38,8 +40,8 @@ class TestDFDLSubset {
 
   @Test def test_groupRefGroupRef() { { runner.runOneTest("groupRefGroupRef") } }
   @Test def test_refInitiator3() { { runner.runOneTest("refInitiator3") } }
-  @Test def test_groupRef() { { runner.runOneTest("groupRef") } }
-  @Test def test_groupRefChoice() { runner.runOneTest("groupRefChoice") }
+  @Test def test_groupRef() { { runnerNV.runOneTest("groupRef") } }
+  @Test def test_groupRefChoice() { runnerNV.runOneTest("groupRefChoice") }
   @Test def test_badGroupRef() { { runner.runOneTest("badGroupRef") } }
   @Test def test_badSeq() { { runner.runOneTest("badSeq") } }
 
diff --git a/daffodil-test/src/test/scala/org/apache/daffodil/section05/facets/TestFacets.scala b/daffodil-test/src/test/scala/org/apache/daffodil/section05/facets/TestFacets.scala
index 8aeb4e0..03d74dd 100644
--- a/daffodil-test/src/test/scala/org/apache/daffodil/section05/facets/TestFacets.scala
+++ b/daffodil-test/src/test/scala/org/apache/daffodil/section05/facets/TestFacets.scala
@@ -26,8 +26,12 @@ object TestFacets {
 
   val testDir = "/org/apache/daffodil/section05/facets/"
   val runner = Runner(testDir, "Facets.tdml", validateTDMLFile = false, validateDFDLSchemas = false)
+  val runnerV = Runner(testDir, "Facets.tdml", validateTDMLFile = false, validateDFDLSchemas = true)
 
-  @AfterClass def tearDown() { runner.reset }
+  @AfterClass def tearDown() {
+    runner.reset
+    runnerV.reset
+  }
 
 }
 
@@ -46,8 +50,8 @@ class TestFacets {
   @Test def test_minMaxInEx16() { runner.runOneTest("minMaxInEx16") }
 
   @Test def test_maxLength07() { runner.runOneTest("maxLength07") }
-  @Test def test_maxLength08() { runner.runOneTest("maxLength08") }
-  @Test def test_maxLength09() { runner.runOneTest("maxLength09") }
+  @Test def test_maxLength08() { runnerV.runOneTest("maxLength08") }
+  @Test def test_maxLength09() { runnerV.runOneTest("maxLength09") }
   @Test def test_maxLength10() { runner.runOneTest("maxLength10") }
   @Test def test_maxLength11() { runner.runOneTest("maxLength11") }
 
@@ -95,16 +99,16 @@ class TestFacets {
   @Test def test_maxLength05() { runner.runOneTest("maxLength05") }
   @Test def test_maxLength06() { runner.runOneTest("maxLength06") }
 
-  @Test def test_totalDigits03() { runner.runOneTest("totalDigits03") }
-  @Test def test_totalDigits04() { runner.runOneTest("totalDigits04") }
-  @Test def test_totalDigits05() { runner.runOneTest("totalDigits05") }
-  @Test def test_totalDigits06() { runner.runOneTest("totalDigits06") }
-  @Test def test_totalDigits07() { runner.runOneTest("totalDigits07") }
-  @Test def test_totalDigits08() { runner.runOneTest("totalDigits08") }
+  @Test def test_totalDigits03() { runnerV.runOneTest("totalDigits03") }
+  @Test def test_totalDigits04() { runnerV.runOneTest("totalDigits04") }
+  @Test def test_totalDigits05() { runnerV.runOneTest("totalDigits05") }
+  @Test def test_totalDigits06() { runnerV.runOneTest("totalDigits06") }
+  @Test def test_totalDigits07() { runnerV.runOneTest("totalDigits07") }
+  @Test def test_totalDigits08() { runnerV.runOneTest("totalDigits08") }
 
-  @Test def test_fractionDigitsFailNeg() { runner.runOneTest("fractionDigitsFailNeg") }
+  @Test def test_fractionDigitsFailNeg() { runnerV.runOneTest("fractionDigitsFailNeg") }
   @Test def test_fractionTotalDigitsFail() { runner.runOneTest("fractionTotalDigitsFail") }
-  @Test def test_fractionDigitsFailNotInt() { runner.runOneTest("fractionDigitsFailNotInt") }
+  @Test def test_fractionDigitsFailNotInt() { runnerV.runOneTest("fractionDigitsFailNotInt") }
 
   @Test def test_arraysMinOccursZero() { runner.runOneTest("arraysMinOccursZero") }
   @Test def test_arraysOccursInRange_01() { runner.runOneTest("arraysOccursInRange_01") }
@@ -209,7 +213,7 @@ class TestFacets {
   @Test def test_totalDigits02() { runner.runOneTest("totalDigits02") }
 
   @Test def test_totalDigits05b() { runner.runOneTest("totalDigits05b") }
-  @Test def test_totalDigits09() { runner.runOneTest("totalDigits09") }
+  @Test def test_totalDigits09() { runnerV.runOneTest("totalDigits09") }
 
   @Test def test_patternRegexDFDL708_01() { runner.runOneTest("patternRegexDFDL708_01") }
   @Test def test_patternRegexDFDL708_02() { runner.runOneTest("patternRegexDFDL708_02") }
diff --git a/daffodil-test/src/test/scala/org/apache/daffodil/section06/namespaces/TestNamespaces.scala b/daffodil-test/src/test/scala/org/apache/daffodil/section06/namespaces/TestNamespaces.scala
index 1b91150..56b9354 100644
--- a/daffodil-test/src/test/scala/org/apache/daffodil/section06/namespaces/TestNamespaces.scala
+++ b/daffodil-test/src/test/scala/org/apache/daffodil/section06/namespaces/TestNamespaces.scala
@@ -27,6 +27,8 @@ object TestNamespaces {
   val testDir = "/org/apache/daffodil/section06/namespaces/"
 
   val runner = Runner(testDir, "namespaces.tdml", validateTDMLFile = true, validateDFDLSchemas = false)
+  val runnerV = Runner(testDir, "namespaces.tdml", validateTDMLFile = true, validateDFDLSchemas = true)
+
   val runner2 = Runner(testDir, "multiFile.tdml", validateTDMLFile = false, validateDFDLSchemas = false)
   val runner3 = Runner(testDir, "includeImport.tdml")
   val runnerWithSchemaValidation = Runner(testDir, "multiFile.tdml", validateTDMLFile = true, validateDFDLSchemas = true)
@@ -61,7 +63,7 @@ class TestNamespaces {
       // Must turn off the Info logging messages, because those will have the filename in them
       // which would create a false positive in this test.
       LoggingDefaults.setLoggingLevel(LogLevel.Warning)
-      runner.runOneTest("combinations_02")
+      runnerV.runOneTest("combinations_02")
     } finally {
       LoggingDefaults.setLoggingLevel(LogLevel.Info)
     }
@@ -92,9 +94,9 @@ class TestNamespaces {
   // DFDL-1204 - this test no longer works. New loader won't accept character U+00B7 as a character
   // in a prefix name.
   // @Test def test_namespaceSpecialChars() { runner.runOneTest("namespaceSpecialChars") }
-  @Test def test_namespaceSpecialChars2() { runner.runOneTest("namespaceSpecialChars2") }
+  @Test def test_namespaceSpecialChars2() { runnerV.runOneTest("namespaceSpecialChars2") }
   @Test def test_namespaceRules1() { runner.runOneTest("namespaceRules1") }
-  @Test def test_namespaceRules2() { runner.runOneTest("namespaceRules2") }
+  @Test def test_namespaceRules2() { runnerV.runOneTest("namespaceRules2") }
 
   @Test def testSimpleIncludeOfFormat() { runner2.runOneTest("simpleInclude") }
   @Test def testSimpleImportOfFormat() { runner2.runOneTest("simpleImport") }
@@ -162,10 +164,10 @@ class TestNamespaces {
   @Test def test_element_conflict_01() { runner.runOneTest("element_conflict_01") }
   @Test def test_element_conflict_02() { runner.runOneTest("element_conflict_02") }
 
-  @Test def test_no_namespace_temp() { runner.runOneTest("no_namespace_temp") }
+  @Test def test_no_namespace_temp() { runnerV.runOneTest("no_namespace_temp") }
 
   @Test def test_lion_eater_ambiguity_01() { runner.runOneTest("lion_eater_ambiguity_01") }
-  @Test def test_lion_eater_ambiguity_01b() { runner.runOneTest("lion_eater_ambiguity_01b") }
+  @Test def test_lion_eater_ambiguity_01b() { runnerV.runOneTest("lion_eater_ambiguity_01b") }
   @Test def test_lion_eater_ambiguity_02() { runner.runOneTest("lion_eater_ambiguity_02") }
   @Test def test_lion_eater_ambiguity_03() { runner.runOneTest("lion_eater_ambiguity_03") }
   @Test def test_lion_eater_ambiguity_04() { runner.runOneTest("lion_eater_ambiguity_04") }
@@ -183,7 +185,7 @@ class TestNamespaces {
   @Test def test_namespace_scope_01() { runner.runOneTest("namespace_scope_01") }
   @Test def test_namespace_scope_02() { runner.runOneTest("namespace_scope_02") }
 
-  @Test def test_error_messages_01() { runner.runOneTest("error_messages_01") }
+  @Test def test_error_messages_01() { runnerV.runOneTest("error_messages_01") }
 
   @Test def test_ibm_format_compat_01() { runner.runOneTest("ibm_format_compat_01") }
   @Test def test_ibm_format_compat_02() { runner.runOneTest("ibm_format_compat_02") }
@@ -191,7 +193,7 @@ class TestNamespaces {
   @Test def test_nonsense_namespace_01() { runner.runOneTest("nonsense_namespace_01") }
   @Test def test_nonsense_namespace_02() { runner.runOneTest("nonsense_namespace_02") }
   @Test def test_nonsense_namespace_03() { runner.runOneTest("nonsense_namespace_03") }
-  @Test def test_nonsense_namespace_04() { runner.runOneTest("nonsense_namespace_04") }
+  @Test def test_nonsense_namespace_04() { runnerV.runOneTest("nonsense_namespace_04") }
 
   @Test def test_junkAnnotation01() { runner.runOneTest("junkAnnotation01") }
 
diff --git a/daffodil-test/src/test/scala/org/apache/daffodil/section07/property_syntax/TestPropertySyntax.scala b/daffodil-test/src/test/scala/org/apache/daffodil/section07/property_syntax/TestPropertySyntax.scala
index e5d4e83..440f09f 100644
--- a/daffodil-test/src/test/scala/org/apache/daffodil/section07/property_syntax/TestPropertySyntax.scala
+++ b/daffodil-test/src/test/scala/org/apache/daffodil/section07/property_syntax/TestPropertySyntax.scala
@@ -27,6 +27,7 @@ object TestPropertySyntax {
 
   val testDir1 = "/org/apache/daffodil/section07/property_syntax/"
   val runner1 = Runner(testDir1, "PropertySyntax.tdml", false, false)
+  val runner1V = Runner(testDir1, "PropertySyntax.tdml", validateTDMLFile = false)
 
   @AfterClass def shutDown {
     runner.reset
@@ -50,7 +51,7 @@ class TestPropertySyntax {
   @Test def test_Lesson3_attribute_form() { runner1.runOneTest("Lesson3_attribute_form") }
   @Test def test_Lesson3_element_form() { runner1.runOneTest("Lesson3_element_form") }
   @Test def test_Lesson3_short_form() { runner1.runOneTest("Lesson3_short_form") }
-  @Test def test_encodingEmptyFail() { runner1.runOneTest("encodingEmptyFail") }
+  @Test def test_encodingEmptyFail() { runner1V.runOneTest("encodingEmptyFail") }
 
   @Test def test_dafProperty1() { runner1.runOneTest("dafProperty1") }
   @Test def test_dafProperty2() { runner1.runOneTest("dafProperty2") }
diff --git a/daffodil-test/src/test/scala/org/apache/daffodil/section23/dfdl_expressions/TestDFDLExpressions.scala b/daffodil-test/src/test/scala/org/apache/daffodil/section23/dfdl_expressions/TestDFDLExpressions.scala
index b498b05..309fc45 100644
--- a/daffodil-test/src/test/scala/org/apache/daffodil/section23/dfdl_expressions/TestDFDLExpressions.scala
+++ b/daffodil-test/src/test/scala/org/apache/daffodil/section23/dfdl_expressions/TestDFDLExpressions.scala
@@ -31,7 +31,8 @@ object TestDFDLExpressions {
 
   val testDir4 = "/org/apache/daffodil/section23/runtime_properties/"
 
-  val runner = Runner(testDir, "expressions.tdml", validateTDMLFile = false, validateDFDLSchemas = false)
+  val runner = Runner(testDir, "expressions.tdml")
+  val runnerNV = Runner(testDir, "expressions.tdml", validateDFDLSchemas = false)
   val runner2 = Runner(testDir2, "Functions.tdml")
   val runner2_utf8 = Runner(testDir2, "Functions_UTF8.tdml")
   val runner2b = Runner(testDir2, "Functions-neg.tdml")
@@ -45,6 +46,7 @@ object TestDFDLExpressions {
   @AfterClass def shutDown() {
     runner4.reset
     runner.reset
+    runnerNV.reset
     runner_fun.reset
     runner2.reset
     runner2_utf8.reset
@@ -94,7 +96,7 @@ class TestDFDLExpressions {
   @Test def test_arrayIndexOutOfBounds_01() { runner.runOneTest("arrayIndexOutOfBounds_01") }
   @Test def test_arrayIndexOutOfBounds_02() { runner.runOneTest("arrayIndexOutOfBounds_02") }
   @Test def test_arrayIndexOutOfBounds_03() { runner.runOneTest("arrayIndexOutOfBounds_03") }
-  
+
   // TODO: TBD DFDL-TICKET, should statically tell this is an invalid index (-1)
   //@Test def test_arrayIndexOutOfBounds_04() { runner.runOneTest("arrayIndexOutOfBounds_04") }
   @Test def test_arrayIndexOutOfBounds_05() { runner.runOneTest("arrayIndexOutOfBounds_05") }
@@ -196,9 +198,9 @@ class TestDFDLExpressions {
   @Test def test_comparison_operators_82() { runner.runOneTest("comparison_operators_82") }
   @Test def test_comparison_operators_83() { runner.runOneTest("comparison_operators_83") }
 
-  @Test def test_regexLookahead() { runner.runOneTest("regexLookahead") }
-  @Test def test_regexLookaheadFail() { runner.runOneTest("regexLookaheadFail") }
-  @Test def test_regexLookaheadFail2() { runner.runOneTest("regexLookaheadFail2") }
+  @Test def test_regexLookahead() { runnerNV.runOneTest("regexLookahead") }
+  @Test def test_regexLookaheadFail() { runnerNV.runOneTest("regexLookaheadFail") }
+  @Test def test_regexLookaheadFail2() { runnerNV.runOneTest("regexLookaheadFail2") }
   //  @Test def test_regexCompatFail() { runner.runOneTest("regexCompatFail") }
 
   @Test def test_expressionRules01() { runner.runOneTest("expressionRules01") }
@@ -250,13 +252,13 @@ class TestDFDLExpressions {
   // DFDL-1043
   // @Test def test_checkConstraintsComplexTypeFails() { runner.runOneTest("checkConstraintsComplexTypeFails") }
 
-  @Test def test_nonFunctionIsDetected() = { runner.runOneTest("nonFunctionIsDetected") }
-  @Test def test_constantFunction1() { runner.runOneTest("constantFunction1") }
-  @Test def test_dfdlPosition1() { runner.runOneTest("dfdlPosition1") }
-  @Test def test_dfdlPosition2() { runner.runOneTest("dfdlPosition2") }
-  @Test def test_dfdlPosition3() { runner.runOneTest("dfdlPosition3") }
-  @Test def test_dfdlPosition4() { runner.runOneTest("dfdlPosition4") }
-  @Test def test_dfdlPosition5() { runner.runOneTest("dfdlPosition5") }
+  @Test def test_nonFunctionIsDetected() = { runnerNV.runOneTest("nonFunctionIsDetected") }
+  @Test def test_constantFunction1() { runnerNV.runOneTest("constantFunction1") }
+  @Test def test_dfdlPosition1() { runnerNV.runOneTest("dfdlPosition1") }
+  @Test def test_dfdlPosition2() { runnerNV.runOneTest("dfdlPosition2") }
+  @Test def test_dfdlPosition3() { runnerNV.runOneTest("dfdlPosition3") }
+  @Test def test_dfdlPosition4() { runnerNV.runOneTest("dfdlPosition4") }
+  @Test def test_dfdlPosition5() { runnerNV.runOneTest("dfdlPosition5") }
 
   @Test def test_repeatFlags1() { runner.runOneTest("repeatFlags1") }
   @Test def test_repeatFlags2() { runner.runOneTest("repeatFlags2") }

-- 
To stop receiving notification emails like this one, please contact
mbeckerle@apache.org.