You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@daffodil.apache.org by sl...@apache.org on 2019/07/02 12:36:38 UTC

[incubator-daffodil] branch master updated: Support terminators and initiators containing %ES;

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

slawrence 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 4b17550  Support terminators and initiators containing %ES;
4b17550 is described below

commit 4b17550d26f73ae3ff168e1297b74f2b6374fb88
Author: Steve Lawrence <sl...@apache.org>
AuthorDate: Thu Jun 27 13:25:21 2019 -0400

    Support terminators and initiators containing %ES;
    
    - Modify DFA building code so that we actually compile %ES; into a DFA
      that matches anything. This allows us to use the same delimiter
      scanning code for finding ES. This also fixes the issue where a literal
      "%ES;" would be unparsed instead of an empty string.
    - Modify the various delimiter scanning iterators so that they skip past
      %ES; delimiters at the appropriate time (e.g. delimiter scanning)
    - Remove the logic in DelimiterParsers to no longer include special
      checks for ES since that is now handled by standard delimiter scanning
      logic and new delimiter iterator logic
    - Update "delimiter not found" error to include all local delimiters
      rather than just the first
    - Update DelimiterNoES cooker to mention that the no ES restriction only
      applies to lengthKind="delimited". This cooker is only used in the
      lengthKind="delimited" case.
    
    DAFFODIL-1474, DAFFODIL-1477, DAFFODIL-1493
---
 .../org/apache/daffodil/CLI/output/output16.txt    |   6 +
 .../grammar/primitives/PrimitivesDelimiters.scala  |  16 +-
 .../apache/daffodil/cookers/EntityReplacer.scala   |   7 +-
 .../apache/daffodil/processors/DFDLDelimiter.scala |  54 +++++-
 .../daffodil/processors/DelimiterIterator.scala    | 137 ++++++++-------
 .../processors/dfa/CreateDelimiterDFA.scala        |   8 +-
 .../org/apache/daffodil/processors/dfa/Rules.scala |  21 +++
 .../apache/daffodil/processors/dfa/Runtime.scala   |   4 +
 .../processors/parsers/DelimitedParsers.scala      |   2 +-
 .../processors/parsers/DelimiterParsers.scala      | 103 ++++-------
 .../daffodil/section06/entities/Entities.tdml      | 188 +++++++++++++++++++--
 .../daffodil/section06/entities/entities_01.tdml   |  20 ++-
 .../sequence_groups/SequenceGroupDelimiters.tdml   |  88 ++++++++++
 .../daffodil/section06/entities/TestEntities.scala |  17 +-
 .../sequence_groups/TestSequenceGroups.scala       |   5 +
 15 files changed, 493 insertions(+), 183 deletions(-)

diff --git a/daffodil-cli/src/it/resources/org/apache/daffodil/CLI/output/output16.txt b/daffodil-cli/src/it/resources/org/apache/daffodil/CLI/output/output16.txt
index b08e952..64ee4e8 100644
--- a/daffodil-cli/src/it/resources/org/apache/daffodil/CLI/output/output16.txt
+++ b/daffodil-cli/src/it/resources/org/apache/daffodil/CLI/output/output16.txt
@@ -10,6 +10,12 @@ byte_entities_6_10
 dataDumpEncoding
 doubleNLseparator
 doubleNLterminator
+emptyStringEntityInitiator_01
+emptyStringEntityInitiator_02
+emptyStringEntityInitiator_03
+emptyStringEntityTermInComplex_01
+emptyStringEntityTermInComplex_02
+emptyStringEntityTermInExpressionDelimited_01
 emptyStringEntityTermInExpression_01
 emptyStringEntityTermInExpression_02
 entityAndNonMix_01
diff --git a/daffodil-core/src/main/scala/org/apache/daffodil/grammar/primitives/PrimitivesDelimiters.scala b/daffodil-core/src/main/scala/org/apache/daffodil/grammar/primitives/PrimitivesDelimiters.scala
index 36f466b..6600381 100644
--- a/daffodil-core/src/main/scala/org/apache/daffodil/grammar/primitives/PrimitivesDelimiters.scala
+++ b/daffodil-core/src/main/scala/org/apache/daffodil/grammar/primitives/PrimitivesDelimiters.scala
@@ -17,19 +17,20 @@
 
 package org.apache.daffodil.grammar.primitives
 
-import org.apache.daffodil.grammar.Terminal
 import org.apache.daffodil.Implicits._;
 import org.apache.daffodil.dsom._
 import org.apache.daffodil.exceptions.Assert
 import org.apache.daffodil.exceptions.ThrowsSDE
 import org.apache.daffodil.grammar.Terminal
-import org.apache.daffodil.processors.parsers.{ Parser => DaffodilParser }
-import org.apache.daffodil.processors.unparsers.{ Unparser => DaffodilUnparser }
+import org.apache.daffodil.grammar.Terminal
 import org.apache.daffodil.processors.dfa.TextParser
 import org.apache.daffodil.processors.parsers.DelimiterTextParser
-import org.apache.daffodil.schema.annotation.props.gen.EscapeKind
 import org.apache.daffodil.processors.parsers.DelimiterTextType
+import org.apache.daffodil.processors.parsers.{ Parser => DaffodilParser }
 import org.apache.daffodil.processors.unparsers.DelimiterTextUnparser
+import org.apache.daffodil.processors.unparsers.{ Unparser => DaffodilUnparser }
+import org.apache.daffodil.schema.annotation.props.gen.EscapeKind
+import org.apache.daffodil.schema.annotation.props.gen.LengthKind
 
 object INoWarn5 { ImplicitsSuppressUnusedImportWarning() }
 
@@ -97,7 +98,12 @@ abstract class DelimiterText(e: Term, eb: Term, delimiterType: DelimiterTextType
 
   lazy val textParser = new TextParser(e.termRuntimeData)
 
-  override lazy val parser: DaffodilParser = new DelimiterTextParser(e.termRuntimeData, textParser, positionalInfo, delimiterType)
+  val isDelimited = e match {
+    case elemB: ElementBase => elemB.lengthKind == LengthKind.Delimited
+    case _ => false
+  }
+
+  override lazy val parser: DaffodilParser = new DelimiterTextParser(e.termRuntimeData, textParser, positionalInfo, delimiterType, isDelimited)
   override lazy val unparser: DaffodilUnparser = new DelimiterTextUnparser(e.termRuntimeData, delimiterType)
 }
 
diff --git a/daffodil-lib/src/main/scala/org/apache/daffodil/cookers/EntityReplacer.scala b/daffodil-lib/src/main/scala/org/apache/daffodil/cookers/EntityReplacer.scala
index 13e492a..80ed58d 100644
--- a/daffodil-lib/src/main/scala/org/apache/daffodil/cookers/EntityReplacer.scala
+++ b/daffodil-lib/src/main/scala/org/apache/daffodil/cookers/EntityReplacer.scala
@@ -800,17 +800,14 @@ class DelimiterCookerNoES(pn: String) extends ListOfString1OrMoreLiteral(pn, tru
   override val oneLiteralCooker: StringLiteralBase =
     new StringLiteralNoCharClassEntities(propName, true) {
 
-      /**
-       * Only ES is disallowed
-       */
       override protected def noCharClassEntities(raw: String, context: ThrowsSDE) {
         // TODO: this isn't quite right, as it will allow combined delimiters
         // that still match the empty string, e.g. "%ES;%WSP*;". We could check
         // if raw.contains("%WSP*;"), but that is too general, preventing valid
         // delimiters like "foo%WSP*;bar". Although the below matches the
         // specification, it's probably not the intended behavior.
-        context.schemaDefinitionUnless(raw != "%WSP*;", "Property dfdl:%s cannot contain %%WSP*;", propName)
-        context.schemaDefinitionUnless(raw != "%ES;", "Property dfdl:%s cannot contain %%ES;", propName)
+        context.schemaDefinitionUnless(raw != "%WSP*;", "Property dfdl:%s cannot contain %%WSP*; when dfdl:lengthKind=\"delimited\".", propName)
+        context.schemaDefinitionUnless(raw != "%ES;", "Property dfdl:%s cannot contain %%ES; when dfdl:lengthKind=\"delimited\".", propName)
       }
     }
 }
diff --git a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DFDLDelimiter.scala b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DFDLDelimiter.scala
index 7b4437e..88104c5 100644
--- a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DFDLDelimiter.scala
+++ b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DFDLDelimiter.scala
@@ -56,6 +56,7 @@ class Delimiter {
   private lazy val WSP = Pattern.compile("(WSP);", Pattern.MULTILINE)
   private lazy val WSP_Plus = Pattern.compile("(WSP\\+);", Pattern.MULTILINE)
   private lazy val WSP_Star = Pattern.compile("(WSP\\*);", Pattern.MULTILINE)
+  private lazy val ES = Pattern.compile("(ES);", Pattern.MULTILINE)
 
   override def toString(): String = {
     return "Delimiter[" + delimiterStr + "]"
@@ -232,6 +233,7 @@ class Delimiter {
                 "|\\u00A0|\\u1680|\\u180E|\\u2000|\\u2001|\\u2002|\\u2003|\\u2004|\\u2005|\\u2006|" +
                 "\\u2007|\\u2008|\\u2009|\\u200A|\\u2028|\\u2029|\\u202F|\\u205F|\\u3000)*")
             } // None or more spaces
+            case ws: ESDelim => // noop
             case char: CharDelim => { // Some character
               char.char match {
                 case '[' => sb.append("\\[")
@@ -264,6 +266,8 @@ class Delimiter {
     val mWSP: Matcher = WSP.matcher(str)
     val mWSP_Plus: Matcher = WSP_Plus.matcher(str)
     val mWSP_Star: Matcher = WSP_Star.matcher(str)
+    val mES: Matcher = ES.matcher(str)
+
     var length: Int = -1
 
     val classList: scala.collection.mutable.Map[String, (Int, Int)] = scala.collection.mutable.Map.empty
@@ -272,6 +276,10 @@ class Delimiter {
       classList += ("NL" -> (mNL.start() -> mNL.end()))
     }
 
+    if (mES.find()) {
+      classList += ("ES" -> (mES.start() -> mES.end()))
+    }
+
     if (mWSP.find()) {
       classList += ("WSP" -> (mWSP.start() -> mWSP.end()))
     }
@@ -292,6 +300,7 @@ class Delimiter {
         case "WSP" => (length, One(new WSPDelim()))
         case "WSP+" => (length, One(new WSPPlusDelim()))
         case "WSP*" => (length, One(new WSPStarDelim()))
+        case "ES" => (length, One(new ESDelim()))
       }
       return result
     }
@@ -381,14 +390,22 @@ class Delimiter {
         idx += 1
       }
     }
-    var resDelimBuf: Array[DelimBase] = null
-    if (numCharClass > 1) {
-      // More than one Char Class, reduction possible!
-      resDelimBuf = reduceDelimBuf(q.toArray[DelimBase])
-    } else {
-      // No need to reduce
-      resDelimBuf = q.toArray[DelimBase]
-    }
+
+    // filter out any %ES; delims, they do not have any effect
+    val delimsNoES = q.filterNot(_.isInstanceOf[ESDelim]).toArray
+
+    val resDelimBuf: Array[DelimBase] =
+      if (delimsNoES.length == 0) {
+        // if the delimiter was just one or more ES's, then make the delim buf
+        // a single ES
+        Array(new ESDelim)
+      } else if (numCharClass > 1) {
+        // More than one Char Class, reduction possible!
+        reduceDelimBuf(delimsNoES)
+      } else {
+        // No need to reduce
+        delimsNoES
+      }
     resDelimBuf
   }
 }
@@ -602,3 +619,24 @@ class WSPStarDelim extends WSPBase with WSP {
 
   def unparseValue(outputNewLine: String): String = ""
 }
+
+/**
+ * This delimiter matches an empty space which effectively is always a
+ * successful match without moving the character position.
+ *
+ * This is useful since it allows us to use the exact same logic for
+ * consuming/matching %ES; as all other delimiters. This is necessary in the
+ * case when dfdl:terminator has %ES; when dfdl:lengthKind != "delimited".
+ * Being able to use the standard delimiter scanning for this makes our lives
+ * much easier and removes lots of special casing. This class establishes the
+ * invariant that there is always a DelimBase object used when "scanning", even
+ * when we're "scanning for nothing".
+ */
+class ESDelim extends DelimBase {
+  override def checkMatch(charIn: Char): Boolean = Assert.impossible("We should never ask if a character matches an %ES;")
+  override def allChars: Seq[Char] = Seq.empty
+  override def print: Unit = {}
+  override def printStr: String = typeName
+  override def typeName: String = "ES"
+  override def unparseValue(outputNewLine: String): String = ""
+}
diff --git a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DelimiterIterator.scala b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DelimiterIterator.scala
index 619b274..e28daed 100644
--- a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DelimiterIterator.scala
+++ b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/DelimiterIterator.scala
@@ -17,85 +17,68 @@
 
 package org.apache.daffodil.processors
 
+import scala.collection.mutable
+
+import org.apache.daffodil.exceptions.Assert
 import org.apache.daffodil.processors.dfa.DFADelimiter
 import org.apache.daffodil.processors.parsers.DelimiterTextType
-import scala.collection.mutable
 import org.apache.daffodil.util.MStackOfInt
 
-trait LocalDelimiters { this: DelimiterIterator =>
-  val delimiterIndexStack: MStackOfInt
-
+trait DelimitersRangeLocal { this: DelimiterIterator =>
   override def reset() {
-    currentIndex = delimiterIndexStack.top - 1
+    currentIndex = this.firstLocalIndex - 1
     indexLimit = this.delimiters.length
   }
 }
 
-trait RemoteDelimiters { this: DelimiterIterator =>
-  val delimiterIndexStack: MStackOfInt
-
+trait DelmitersRangeRemote { this: DelimiterIterator =>
   override def reset() {
     currentIndex = -1
-    indexLimit = delimiterIndexStack.top
+    indexLimit = this.firstLocalIndex
   }
 }
 
-trait RemoteAndLocalDelimiters { this: DelimiterIterator =>
+trait DelimitersRangeAll { this: DelimiterIterator =>
   override def reset() {
     currentIndex = -1
     indexLimit = this.delimiters.length
   }
 }
 
-trait TypedDelimiter { this: DelimiterIterator =>
-  protected def delimType: DelimiterTextType.Type
-
-  override def isOfInterest(delim: DFADelimiter): Boolean = {
-    delim.delimType == delimType
-  }
-}
-
-trait TerminatingMarkup { this: DelimiterIterator =>
-  override def isOfInterest(delim: DFADelimiter) = {
-    delim.delimType == DelimiterTextType.Terminator || delim.delimType == DelimiterTextType.Separator
-  }
-}
-
-trait Separators { this: DelimiterIterator =>
-  @inline
-  override def isOfInterest(delim: DFADelimiter) = {
-    delim.delimType == DelimiterTextType.Separator
-  }
-}
-
-trait Initiators { this: DelimiterIterator =>
-  @inline
-  override def isOfInterest(delim: DFADelimiter) = {
-    delim.delimType == DelimiterTextType.Initiator
-  }
-}
-
-trait Terminators { this: DelimiterIterator =>
-  @inline
-  override def isOfInterest(delim: DFADelimiter) = {
-    delim.delimType == DelimiterTextType.Terminator
-  }
-}
-
+/**
+ * It's important to note that this does not follow standard iterator
+ * practices.
+ *
+ * First, reset() must always be called before iteration beings. This is
+ * because this function initializes the indices so that only the correct
+ * proper range of delimiters are scanned for. Without calling this, the
+ * iterator will return nothing.
+ *
+ * Second, hasNext() is what moves the iterator to the next delimiter, not
+ * next(). next() simply returns the current index in the iterator. So calling
+ * next() twice in a row will return the same thing. Calling hasNext before
+ * calling next is standard iterator practice, though, and how we always use
+ * it, but it is actually required.
+ *
+ * So this DelmiterIterators should always be used like so:
+ *
+ *   val iter = new DelimiterIterator()
+ *   iter.reset()
+ *   while (iter.hasNext()) {
+ *     val delim = iter.next()
+ *     ...
+ *   }
+ *
+ */
 abstract class DelimiterIterator(val delimiters: mutable.ArrayBuffer[DFADelimiter]) {
 
   protected def isOfInterest(delim: DFADelimiter): Boolean
+  def firstLocalIndex: Int
   def reset(): Unit
 
   var currentIndex = -1
   var indexLimit = -1
 
-  // TODO: I'm a little concernece that this doesn't follow standard iterator
-  // practicies. For exmaple, if you called di.next() twice in a row without
-  // calling hasNext, the second next() call would give you the same delimiter.
-  // So for this to work, you really do need to call hasNext before ever
-  // calling next. Although one should follow that practice with iterators, it
-  // isn't usually required.
   def hasNext(): Boolean = {
     currentIndex += 1
     while (currentIndex >= 0 && currentIndex < indexLimit) {
@@ -110,30 +93,40 @@ abstract class DelimiterIterator(val delimiters: mutable.ArrayBuffer[DFADelimite
   def next(): DFADelimiter = {
     delimiters(currentIndex)
   }
+
+  def isRemote(): Boolean = currentIndex < firstLocalIndex
 }
 
-class AllTerminatingMarkupDelimiterIterator(d: mutable.ArrayBuffer[DFADelimiter])
+class AllTerminatingMarkupDelimiterIterator(d: mutable.ArrayBuffer[DFADelimiter], override val firstLocalIndex: Int)
   extends DelimiterIterator(d)
-  with RemoteAndLocalDelimiters
-  with TerminatingMarkup
+  with DelimitersRangeAll {
 
-class LocalTypedDelimiterIterator(override val delimType: DelimiterTextType.Type, d: mutable.ArrayBuffer[DFADelimiter], override val delimiterIndexStack: MStackOfInt)
-  extends DelimiterIterator(d)
-  with LocalDelimiters
-  with TypedDelimiter
+  override def isOfInterest(delim: DFADelimiter): Boolean = {
+    // include only term/sep, ignore all ES
+    if (delim.isES) false
+    else delim.delimType == DelimiterTextType.Terminator || delim.delimType == DelimiterTextType.Separator
+  }
+}
 
-class RemoteTypedDelimiterIterator(override val delimType: DelimiterTextType.Type, d: mutable.ArrayBuffer[DFADelimiter], override val delimiterIndexStack: MStackOfInt)
+class LocalTypedDelimiterIterator(delimType: DelimiterTextType.Type, d: mutable.ArrayBuffer[DFADelimiter], override val firstLocalIndex: Int)
   extends DelimiterIterator(d)
-  with RemoteDelimiters
-  with TypedDelimiter
+  with DelimitersRangeLocal {
+
+  override def isOfInterest(delim: DFADelimiter): Boolean = {
+    // include ES's, the DelmiterRangeLocal will ensure we ignore remote delims
+    delim.delimType == delimType
+  }
+}
 
-class RemoteTerminatingMarkupAndLocalTypedDelimiterIterator(localType: DelimiterTextType.Type, d: mutable.ArrayBuffer[DFADelimiter], delimiterIndexStack: MStackOfInt)
-    extends DelimiterIterator(d)
-    with RemoteAndLocalDelimiters {
+class RemoteTerminatingMarkupAndLocalTypedDelimiterIterator(localType: DelimiterTextType.Type, d: mutable.ArrayBuffer[DFADelimiter], override val firstLocalIndex: Int)
+  extends DelimiterIterator(d)
+  with DelimitersRangeAll {
 
   override def isOfInterest(delim: DFADelimiter): Boolean = {
-    if (currentIndex < delimiterIndexStack.top) {
-      delim.delimType == DelimiterTextType.Terminator || delim.delimType == DelimiterTextType.Separator
+    // all remote term/sep excluding ES, or all local typed including ES
+    if (isRemote()) {
+      if (delim.isES) false
+      else delim.delimType == DelimiterTextType.Terminator || delim.delimType == DelimiterTextType.Separator
     } else {
       delim.delimType == localType
     }
@@ -141,8 +134,14 @@ class RemoteTerminatingMarkupAndLocalTypedDelimiterIterator(localType: Delimiter
 }
 
 class AllDelimiterIterator(d: mutable.ArrayBuffer[DFADelimiter])
-    extends DelimiterIterator(d)
-    with RemoteAndLocalDelimiters {
+  extends DelimiterIterator(d)
+  with DelimitersRangeAll {
 
-  override def isOfInterest(delim: DFADelimiter) = true
+  override val firstLocalIndex: Int = 0
+  override def isRemote(): Boolean = Assert.usageError("Not be used, no concept of local/remote here")
+
+  override def isOfInterest(delim: DFADelimiter) = {
+    // all delimiters except for ES
+    !delim.isES
+  }
 }
diff --git a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/dfa/CreateDelimiterDFA.scala b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/dfa/CreateDelimiterDFA.scala
index ceb3a97..8ae4a61 100644
--- a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/dfa/CreateDelimiterDFA.scala
+++ b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/dfa/CreateDelimiterDFA.scala
@@ -19,16 +19,17 @@ package org.apache.daffodil.processors.dfa
 
 import scala.collection.mutable.ArrayBuffer
 
+import org.apache.daffodil.exceptions.Assert
 import org.apache.daffodil.processors.CharDelim
 import org.apache.daffodil.processors.DelimBase
 import org.apache.daffodil.processors.Delimiter
+import org.apache.daffodil.processors.ESDelim
 import org.apache.daffodil.processors.NLDelim
+import org.apache.daffodil.processors.RuntimeData
 import org.apache.daffodil.processors.WSPDelim
 import org.apache.daffodil.processors.WSPPlusDelim
 import org.apache.daffodil.processors.WSPStarDelim
-import org.apache.daffodil.processors.RuntimeData
 import org.apache.daffodil.processors.parsers.DelimiterTextType
-import org.apache.daffodil.exceptions.Assert
 
 object CreateDelimiterDFA {
 
@@ -119,6 +120,9 @@ object CreateDelimiterDFA {
       case d: NLDelim => {
         new NLState(allStates, nextState, stateNum)
       }
+      case d: ESDelim => {
+        new ESState(allStates, nextState, stateNum)
+      }
     }
     theState
   }
diff --git a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/dfa/Rules.scala b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/dfa/Rules.scala
index c159d2c..561dfae 100644
--- a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/dfa/Rules.scala
+++ b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/dfa/Rules.scala
@@ -735,3 +735,24 @@ class NLState(states: => ArrayBuffer[State], val nextState: Int, val stateNum: I
 
   def checkMatch(charIn: Char): Boolean = isNLNotCR(charIn) || isCR(charIn)
 }
+
+
+class ESState(states: => ArrayBuffer[State], val nextState: Int, val stateNum: Int)
+  extends NLBase(states) {
+
+  stateName = "ESState"
+
+  /**
+   * This contains a sortof degenerate rule that always succeeds to handle the
+   * empty-string corner case required by the %ES; character class. This will
+   * immediately match while consuming no characters.
+   */
+  lazy val rules = ArrayBuffer(
+    Rule { (r: Registers) => { true } } { (r: Registers) =>
+      {
+        r.status = StateKind.Succeeded
+      }
+    }
+  )
+  def checkMatch(charIn: Char): Boolean = Assert.impossible("We should never ask if a character matches an %ES;")
+}
diff --git a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/dfa/Runtime.scala b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/dfa/Runtime.scala
index b825d6e..ab0fa8d 100644
--- a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/dfa/Runtime.scala
+++ b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/dfa/Runtime.scala
@@ -134,6 +134,10 @@ trait DFADelimiter extends DFA {
 
   final override def run(r: Registers): Unit = runLoop(r, DFA.FinalState, StateKind.Succeeded)
 
+  // We frequently want to know if a delimiter is ES or not when iterating over
+  // delimiters, so cache the result of this comparison
+  final val isES = lookingFor == "%ES;"
+
   def unparseValue: String
 }
 
diff --git a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/DelimitedParsers.scala b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/DelimitedParsers.scala
index 38b961a..f97112c 100644
--- a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/DelimitedParsers.scala
+++ b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/DelimitedParsers.scala
@@ -71,7 +71,7 @@ class StringDelimitedParser(
     //      gram.checkDelimiterDistinctness(esObj.escapeSchemeKind, optPadChar, finalOptEscChar,
     //        finalOptEscEscChar, optEscBlkStart, optEscBlkEnd, delimsCooked, postEscapeSchemeEvalState)
 
-    val delimIter = new AllTerminatingMarkupDelimiterIterator(start.mpstate.delimiters)
+    val delimIter = new AllTerminatingMarkupDelimiterIterator(start.mpstate.delimiters, start.mpstate.delimitersLocalIndexStack.top)
     val fieldDFA = fieldDFAEv.evaluate(start)
 
     start.clearDelimitedParseResult
diff --git a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/DelimiterParsers.scala b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/DelimiterParsers.scala
index efa09c8..b9e68fc 100644
--- a/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/DelimiterParsers.scala
+++ b/daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/DelimiterParsers.scala
@@ -24,7 +24,6 @@ import org.apache.daffodil.util.Enum
 import org.apache.daffodil.processors.TermRuntimeData
 import org.apache.daffodil.processors.DelimiterIterator
 import org.apache.daffodil.processors.LocalTypedDelimiterIterator
-import org.apache.daffodil.processors.RemoteTypedDelimiterIterator
 import org.apache.daffodil.processors.RemoteTerminatingMarkupAndLocalTypedDelimiterIterator
 import org.apache.daffodil.util.Maybe
 import org.apache.daffodil.util.Maybe._
@@ -43,7 +42,8 @@ class DelimiterTextParser(
   rd: TermRuntimeData,
   textParser: TextParser,
   positionalInfo: String,
-  delimiterType: DelimiterTextType.Type)
+  delimiterType: DelimiterTextType.Type,
+  isDelimited: Boolean)
   extends TextPrimParser {
 
   override lazy val runtimeDependencies = rd.encodingInfo.runtimeDependencies
@@ -74,57 +74,20 @@ class DelimiterTextParser(
     return false
   }
 
-  private def findES(delimIter: DelimiterIterator): Maybe[DFADelimiter] = {
-    delimIter.reset()
-    while (delimIter.hasNext()) {
-      val d = delimIter.next()
-      if (d.lookingFor == "%ES;") {
-        return One(d)
-      }
-    }
-    Nope
-  }
-
-  private def findLocalES(state: PState): Maybe[DFADelimiter] = {
-    val delimIter = new LocalTypedDelimiterIterator(delimiterType, state.mpstate.delimiters, state.mpstate.delimitersLocalIndexStack)
-    findES(delimIter)
-  }
+  override def parse(start: PState): Unit = {
 
-  private def findRemoteES(state: PState): Maybe[DFADelimiter] = {
-    val delimIter = new RemoteTypedDelimiterIterator(delimiterType, state.mpstate.delimiters, state.mpstate.delimitersLocalIndexStack)
-    findES(delimIter)
-  }
+    val maybeDelimIter =
+      if (delimiterType == DelimiterTextType.Terminator && !isDelimited) {
+        Maybe(new LocalTypedDelimiterIterator(delimiterType, start.mpstate.delimiters, start.mpstate.delimitersLocalIndexStack.top))
+      } else if (delimiterType == DelimiterTextType.Initiator || !start.delimitedParseResult.isDefined) {
+        Maybe(new RemoteTerminatingMarkupAndLocalTypedDelimiterIterator(delimiterType, start.mpstate.delimiters, start.mpstate.delimitersLocalIndexStack.top))
+      } else {
+        Nope
+      }
 
-  override def parse(start: PState): Unit = {
     val foundDelimiter =
-      if (delimiterType == DelimiterTextType.Initiator || !start.delimitedParseResult.isDefined) {
-        //
-        // We are going to scan for delimiter text.
-        //
-        // FIXME: This is incorrect. It grabs all local delimiters even if we're last in a group so the
-        // prefix or infix separator cannot be relevant, or if we're in the middle of a group with required elements following so
-        // the sequence's (or any enclosing sequence's) terminator cannot be relevant.
-        //
-        // Fixing this is going to require the compiler to pre-compute the relevant delimiters for every
-        // Term. (relevant delimiters meaning the specific compiled expressions that are relevant.)
-        //
-        // TODO: PERFORMANCE: this should also help performance by eliminating the construction of lists/sets of
-        // these things at run time. We should try to not allocate objects here.
-        //
-        // FIXME: This is incorrect. It is going to get too many delimiters. See above.
-        // Nowever, code below does assume that we always find a match, even to incorrect markup, so fixing this is
-        // more than just getting the right set of remote delims here.
-        //
-        // Note: It isn't even as simple as precalculating all the delimiters
-        // (which isn't easy to start with). For example, sometimes we do not
-        // know if we're at the last in the group until runtime due to an
-        // expression that determines occursCount. So this functions needs to
-        // make a runtime calculation to determine if it is the last in a
-        // group, which might be difficult to do.
-        val delimIter = new RemoteTerminatingMarkupAndLocalTypedDelimiterIterator(delimiterType, start.mpstate.delimiters, start.mpstate.delimitersLocalIndexStack)
-
-        val result = textParser.parse(start, start.dataInputStream, delimIter, true)
-        result
+      if (maybeDelimIter.isDefined) {
+        textParser.parse(start, start.dataInputStream, maybeDelimIter.get, true)
       } else {
         start.delimitedParseResult
       }
@@ -144,30 +107,24 @@ class DelimiterTextParser(
       Assert.invariant(wasDelimiterTextSkipped)
       start.clearDelimitedParseResult()
     } else {
-      // Did not find delimiter.
-      //
-      // Still can be ok if ES is a delimiter. That's allowed when we're not lengthKind='delimited'
-      // That is, it is allowed if the delimiter is just part of the data syntax, but is not being
-      // used to determine length.
-      if (findLocalES(start).isDefined) {
-        // found local ES, nothing to consume
-        return
-      } else {
-        val rES = findRemoteES(start)
-        if (rES.isDefined) {
-          // has remote but not local (PE)
-          PE(start, "Found out of scope delimiter: %ES;")
-          return
-        } else {
-          //
-          // no match and no ES in delims
-          //
-          val optDelim = start.mpstate.delimiters.find { d => d.delimType == delimiterType }
-          Assert.invariant(optDelim.isDefined)
-          PE(start, "%s '%s' not found.", delimiterType.toString, optDelim.get.lookingFor)
-          return
-        }
+      // no match found, gather up the local typed delims for an error message
+
+      val scannedDelims = maybeDelimIter.get
+      scannedDelims.reset()
+
+      // skip remote delims
+      while (scannedDelims.hasNext() && scannedDelims.isRemote) {
+        // do nothing, hasNext will increment the iterator
+      }
+
+      // gather local typed delims 
+      var localTypedDelims = scannedDelims.next().lookingFor
+      while (scannedDelims.hasNext()) {
+        localTypedDelims = localTypedDelims + " " + scannedDelims.next().lookingFor
       }
+
+      PE(start, "%s '%s' not found", delimiterType.toString, localTypedDelims)
+      return
     }
   }
 }
diff --git a/daffodil-test/src/test/resources/org/apache/daffodil/section06/entities/Entities.tdml b/daffodil-test/src/test/resources/org/apache/daffodil/section06/entities/Entities.tdml
index 0025d3a..b128c33 100644
--- a/daffodil-test/src/test/resources/org/apache/daffodil/section06/entities/Entities.tdml
+++ b/daffodil-test/src/test/resources/org/apache/daffodil/section06/entities/Entities.tdml
@@ -1129,17 +1129,17 @@ is multiple bytes in UTF-8 encoding that is used"
     <xs:element name="root" dfdl:lengthKind="implicit">
       <xs:complexType>
 
-        <xs:group ref="ex:tString50" />
+        <xs:group ref="ex:tString10" />
 
       </xs:complexType>
     </xs:element>
     
-    <xs:group name="tString50">
+    <xs:group name="tString10">
       <xs:sequence>
         <xs:element name="value" type="xs:string" dfdl:lengthKind="pattern"
-          dfdl:lengthPattern="[^END]{0,49}(?=END)|.{50}" />
+          dfdl:lengthPattern="[^END]{0,9}(?=END)|.{10}" />
           <xs:sequence
-          dfdl:terminator="{if (fn:string-length(./ex:value) eq 50) then '%ES; %DEL;' else 'END'}" />
+          dfdl:terminator="{if (fn:string-length(./ex:value) eq 10) then '%ES;' else 'END'}" />
           <!-- %DEL is there because %ES cannot be alone in list. -->
       </xs:sequence>
     </xs:group>
@@ -1150,38 +1150,200 @@ is multiple bytes in UTF-8 encoding that is used"
     Test Name: emptyStringEntityTermInExpression_01
        Schema: EmptyStringEntityInExpression
       Purpose: This test demonstrates the ability for the %ES; entity to be used in an expression.
-               In this case, the value is 50 characters long so %ES; is a terminator.
+               In this case, the value is 10 characters long so %ES; is a terminator.
   -->
 
   <tdml:parserTestCase name="emptyStringEntityTermInExpression_01" root="root" model="EmptyStringEntityInExpression" description="">
     <tdml:document>
-      <tdml:documentPart type="text"><![CDATA[01234567890123456789012345678901234567890123456789]]></tdml:documentPart>
+      <tdml:documentPart type="text"><![CDATA[0123456789]]></tdml:documentPart>
     </tdml:document>
-    <tdml:errors>
-      <tdml:error>Illegal</tdml:error>
-      <tdml:error>%ES;</tdml:error>
-    </tdml:errors>
+    <tdml:infoset>
+      <tdml:dfdlInfoset>
+        <ex:root>
+          <ex:value>0123456789</ex:value>
+        </ex:root>
+      </tdml:dfdlInfoset>
+    </tdml:infoset>
   </tdml:parserTestCase>
   
   <!--
     Test Name: emptyStringEntityTermInExpression_02
        Schema: EmptyStringEntityInExpression
       Purpose: This test demonstrates the ability for the %ES; entity to be used in an expression.
-               In this case, the value is NOT 50 characters long so %ES; is NOT a terminator.
+               In this case, the value is NOT 10 characters long so %ES; is NOT a terminator.
   -->
 
   <tdml:parserTestCase name="emptyStringEntityTermInExpression_02" root="root" model="EmptyStringEntityInExpression"
     description="">
     <tdml:document>
-      <tdml:documentPart type="text"><![CDATA[012345678901234567890123456789012345678901END]]></tdml:documentPart>
+      <tdml:documentPart type="text"><![CDATA[01234END]]></tdml:documentPart>
+    </tdml:document>
+    <tdml:infoset>
+      <tdml:dfdlInfoset>
+        <ex:root>
+          <ex:value>01234</ex:value>
+        </ex:root>
+      </tdml:dfdlInfoset>
+    </tdml:infoset>
+  </tdml:parserTestCase>
+
+  <tdml:defineSchema name="EmptyStringEntityInExpressionDelimited">
+  
+    <xs:include schemaLocation="org/apache/daffodil/xsd/DFDLGeneralFormat.dfdl.xsd"/>
+    <dfdl:format ref="ex:GeneralFormat" representation="text" encoding="US-ASCII" 
+      separator="" byteOrder="bigEndian" ignoreCase="no" textNumberRep="standard"
+      initiator="" initiatedContent="no" terminator="" />
+
+    <xs:element name="root" dfdl:lengthKind="implicit">
+      <xs:complexType>
+        <xs:sequence>
+          <xs:element name="key" type="xs:string" dfdl:lengthKind="explicit" dfdl:length="1" />
+          <xs:element name="value" type="xs:string" dfdl:lengthKind="delimited"
+            dfdl:terminator="{ if (../ex:key eq '1') then '%ES;' else 'END' }" />
+        </xs:sequence>
+      </xs:complexType>
+    </xs:element>
+
+  </tdml:defineSchema>
+
+  <!--
+    Test Name: emptyStringEntityTermInExpressionDelimited_01
+       Schema: EmptyStringEntityInExpressionDelimited
+      Purpose: This test demonstrates the that a terminator cannot contain %ES; if lengthKind is delimited.
+  -->
+
+  <tdml:parserTestCase name="emptyStringEntityTermInExpressionDelimited_01" root="root" model="EmptyStringEntityInExpressionDelimited"
+    description="">
+    <tdml:document>
+      <tdml:documentPart type="text"><![CDATA[1value]]></tdml:documentPart>
+    </tdml:document>
+    <tdml:errors>
+      <tdml:error>Schema Definition Error</tdml:error>
+      <tdml:error>dfdl:terminator</tdml:error>
+      <tdml:error>cannot contain %ES;</tdml:error>
+      <tdml:error>dfdl:lengthKind="delimited"</tdml:error>
+    </tdml:errors>
+  </tdml:parserTestCase>
+
+  <tdml:defineSchema name="EmptyStringEntityTermInComplex">
+  
+    <xs:include schemaLocation="org/apache/daffodil/xsd/DFDLGeneralFormat.dfdl.xsd"/>
+    <dfdl:format ref="ex:GeneralFormat" representation="text" encoding="US-ASCII" 
+      separator="" byteOrder="bigEndian" ignoreCase="no" textNumberRep="standard"
+      initiator="" initiatedContent="no" terminator="" />
+
+    <xs:element name="root" dfdl:lengthKind="explicit" dfdl:length="10" dfdl:terminator="%ES; END">
+      <xs:complexType>
+        <xs:sequence dfdl:separator=",">
+          <xs:element name="value" type="xs:string" dfdl:lengthKind="delimited" maxOccurs="unbounded" />
+        </xs:sequence>
+      </xs:complexType>
+    </xs:element>
+
+  </tdml:defineSchema>
+
+  <!--
+    Test Name: emptyStringEntityTermInComplex_01
+       Schema: EmptyStringEntityInExpressionDelimited
+      Purpose: This test demonstrates that a fixed length complex with ES terminator works, even if children are delimited
+  -->
+
+  <tdml:parserTestCase name="emptyStringEntityTermInComplex_01" root="root" model="EmptyStringEntityTermInComplex"
+    description="">
+    <tdml:document>
+      <tdml:documentPart type="text"><![CDATA[1,3,5,7,10]]></tdml:documentPart>
+    </tdml:document>
+    <tdml:infoset>
+      <tdml:dfdlInfoset>
+        <ex:root>
+          <ex:value>1</ex:value>
+          <ex:value>3</ex:value>
+          <ex:value>5</ex:value>
+          <ex:value>7</ex:value>
+          <ex:value>10</ex:value>
+        </ex:root>
+      </tdml:dfdlInfoset>
+    </tdml:infoset>
+  </tdml:parserTestCase>
+
+  <!--
+    Test Name: emptyStringEntityTermInComplex_02
+       Schema: EmptyStringEntityInExpressionDelimited
+      Purpose: This test demonstrates that a fixed length complex with ES terminator works, even if children are delimited
+  -->
+
+  <tdml:parserTestCase name="emptyStringEntityTermInComplex_02" root="root" model="EmptyStringEntityTermInComplex"
+    description="" roundTrip="twoPass">
+    <tdml:document>
+      <tdml:documentPart type="text"><![CDATA[1,3,5,7,10END]]></tdml:documentPart>
+    </tdml:document>
+    <tdml:infoset>
+      <tdml:dfdlInfoset>
+        <ex:root>
+          <ex:value>1</ex:value>
+          <ex:value>3</ex:value>
+          <ex:value>5</ex:value>
+          <ex:value>7</ex:value>
+          <ex:value>10</ex:value>
+        </ex:root>
+      </tdml:dfdlInfoset>
+    </tdml:infoset>
+  </tdml:parserTestCase>
+
+  <tdml:defineSchema name="EmptyStringEntityInitiator">
+  
+    <xs:include schemaLocation="org/apache/daffodil/xsd/DFDLGeneralFormat.dfdl.xsd"/>
+    <dfdl:format ref="ex:GeneralFormat" representation="text" encoding="US-ASCII" 
+      separator="" byteOrder="bigEndian" ignoreCase="no" textNumberRep="standard"
+      initiator="" initiatedContent="no" terminator="" />
+
+    <xs:element name="root" dfdl:terminator="END">
+      <xs:complexType>
+        <xs:sequence >
+          <xs:element name="value" type="xs:string" dfdl:initiator="%ES; DATA=" dfdl:lengthKind="delimited" minOccurs="0" />
+        </xs:sequence>
+      </xs:complexType>
+    </xs:element>
+
+  </tdml:defineSchema>
+
+  <tdml:parserTestCase name="emptyStringEntityInitiator_01" root="root" model="EmptyStringEntityInitiator"
+    description="">
+    <tdml:document>
+      <tdml:documentPart type="text"><![CDATA[dataEND]]></tdml:documentPart>
     </tdml:document>
     <tdml:infoset>
       <tdml:dfdlInfoset>
         <ex:root>
-          <ex:value>012345678901234567890123456789012345678901</ex:value>
+          <ex:value>data</ex:value>
         </ex:root>
       </tdml:dfdlInfoset>
+    </tdml:infoset>
+  </tdml:parserTestCase>
 
+  <tdml:parserTestCase name="emptyStringEntityInitiator_02" root="root" model="EmptyStringEntityInitiator"
+    description="" roundTrip="twoPass">
+    <tdml:document>
+      <tdml:documentPart type="text"><![CDATA[DATA=dataEND]]></tdml:documentPart>
+    </tdml:document>
+    <tdml:infoset>
+      <tdml:dfdlInfoset>
+        <ex:root>
+          <ex:value>data</ex:value>
+        </ex:root>
+      </tdml:dfdlInfoset>
+    </tdml:infoset>
+  </tdml:parserTestCase>
+
+  <tdml:parserTestCase name="emptyStringEntityInitiator_03" root="root" model="EmptyStringEntityInitiator"
+    description="">
+    <tdml:document>
+      <tdml:documentPart type="text"><![CDATA[END]]></tdml:documentPart>
+    </tdml:document>
+    <tdml:infoset>
+      <tdml:dfdlInfoset>
+        <ex:root />
+      </tdml:dfdlInfoset>
     </tdml:infoset>
   </tdml:parserTestCase>
 
diff --git a/daffodil-test/src/test/resources/org/apache/daffodil/section06/entities/entities_01.tdml b/daffodil-test/src/test/resources/org/apache/daffodil/section06/entities/entities_01.tdml
index 25c3609..21b0868 100644
--- a/daffodil-test/src/test/resources/org/apache/daffodil/section06/entities/entities_01.tdml
+++ b/daffodil-test/src/test/resources/org/apache/daffodil/section06/entities/entities_01.tdml
@@ -89,7 +89,7 @@
       <xs:complexType>
         <xs:sequence dfdl:separator=",">
           <xs:element name="a" type="xs:int" dfdl:lengthKind="explicit"
-            dfdl:length="1" dfdl:terminator=": %ES;" />
+            dfdl:length="1" dfdl:terminator="%ES; :" />
           <xs:element name="b" type="xs:int" dfdl:lengthKind="explicit"
             dfdl:length="1" />
         </xs:sequence>
@@ -97,7 +97,7 @@
     </xs:element>
   </tdml:defineSchema>
 
-  <tdml:parserTestCase name="entity_fail_03" root="root"
+  <tdml:parserTestCase name="entity_fail_03a" root="root"
     model="sch3"
     description="Terminator cannot contain ES. %ES; is used only in nilValue - DFDL-6-046R
     Note: Revised - DFDL Spec. now allows ES for Initiators as long as it is not the only entry in the list.">
@@ -112,6 +112,22 @@
     </tdml:infoset>
   </tdml:parserTestCase>
 
+  <tdml:parserTestCase name="entity_fail_03b" root="root"
+    model="sch3"
+    roundTrip="twoPass"
+    description="Terminator cannot contain ES. %ES; is used only in nilValue - DFDL-6-046R
+    Note: Revised - DFDL Spec. now allows ES for Initiators as long as it is not the only entry in the list.">
+    <tdml:document>0:,1</tdml:document>
+     <tdml:infoset>
+      <tdml:dfdlInfoset>
+        <ex:root>
+          <a>0</a>
+          <b>1</b>
+        </ex:root>
+      </tdml:dfdlInfoset>
+    </tdml:infoset>
+  </tdml:parserTestCase>
+
   <tdml:defineSchema name="sch4">
     <xs:include schemaLocation="org/apache/daffodil/xsd/DFDLGeneralFormat.dfdl.xsd"/>
     <dfdl:format ref="ex:GeneralFormat" initiator=""
diff --git a/daffodil-test/src/test/resources/org/apache/daffodil/section14/sequence_groups/SequenceGroupDelimiters.tdml b/daffodil-test/src/test/resources/org/apache/daffodil/section14/sequence_groups/SequenceGroupDelimiters.tdml
index c29cdac..7e71430 100644
--- a/daffodil-test/src/test/resources/org/apache/daffodil/section14/sequence_groups/SequenceGroupDelimiters.tdml
+++ b/daffodil-test/src/test/resources/org/apache/daffodil/section14/sequence_groups/SequenceGroupDelimiters.tdml
@@ -526,4 +526,92 @@
 
   </tdml:parserTestCase>
 
+  <tdml:defineSchema name="delimiterScanning">
+
+    <xs:include schemaLocation="org/apache/daffodil/xsd/DFDLGeneralFormat.dfdl.xsd"/>
+    <dfdl:format ref="ex:GeneralFormat" lengthKind="implicit" separatorPosition="infix" />
+
+    <xs:element name="explicit">
+      <xs:complexType>
+        <xs:sequence dfdl:separator=";">
+            <xs:element name="foo" type="xs:string" dfdl:lengthKind="explicit" dfdl:length="3" />
+            <xs:sequence dfdl:separator =",">
+              <xs:element name="bar" type="xs:string" dfdl:lengthKind="explicit" dfdl:length="3" />
+              <xs:element name="baz" type="xs:string" dfdl:lengthKind="explicit" dfdl:length="3" />
+            </xs:sequence>
+           <xs:element name="qux" type="xs:string" dfdl:lengthKind="explicit" dfdl:length="3" />
+        </xs:sequence>
+      </xs:complexType>
+    </xs:element>
+
+    <xs:element name="delimited">
+      <xs:complexType>
+        <xs:sequence dfdl:separator=";">
+            <xs:element name="foo" type="xs:string" dfdl:lengthKind="delimited" />
+            <xs:sequence dfdl:separator =",">
+              <xs:element name="bar" type="xs:string" dfdl:lengthKind="delimited" />
+              <xs:element name="baz" type="xs:string" dfdl:lengthKind="delimited" />
+            </xs:sequence>
+           <xs:element name="qux" type="xs:string" dfdl:lengthKind="delimited" />
+        </xs:sequence>
+      </xs:complexType>
+    </xs:element>
+
+  </tdml:defineSchema>
+
+  <tdml:parserTestCase name="delimiterScanning_01" root="explicit"
+    model="delimiterScanning" description="test for correct delimiter scanning">
+
+    <tdml:document>
+      <tdml:documentPart type="text"><![CDATA[foo;bar,b,z;qux]]></tdml:documentPart>
+    </tdml:document>
+
+    <tdml:infoset>
+      <tdml:dfdlInfoset>
+        <tns:explicit>
+          <tns:foo>foo</tns:foo>
+          <tns:bar>bar</tns:bar>
+          <tns:baz>b,z</tns:baz>
+          <tns:qux>qux</tns:qux>
+        </tns:explicit>
+      </tdml:dfdlInfoset>
+    </tdml:infoset>
+
+  </tdml:parserTestCase>
+
+  <tdml:parserTestCase name="delimiterScanning_02" root="explicit"
+    model="delimiterScanning" description="test for correct delimiter scanning">
+
+    <tdml:document>
+      <tdml:documentPart type="text"><![CDATA[foo;bar,b,z,qux]]></tdml:documentPart>
+    </tdml:document>
+
+    <tdml:errors>
+      <tdml:error>Parse Error</tdml:error>
+      <tdml:error>Failed to parse infix separator</tdml:error>
+      <tdml:error>Separator ';' not found</tdml:error>
+    </tdml:errors>
+
+  </tdml:parserTestCase>
+
+  <tdml:parserTestCase name="delimiterScanning_03" root="delimited"
+    model="delimiterScanning" description="test for correct delimiter scanning. We should not scan for or find the second comma since bar is the last in the infix separated sequence">
+
+    <tdml:document>
+      <tdml:documentPart type="text"><![CDATA[foo;bar,b,z;qux]]></tdml:documentPart>
+    </tdml:document>
+
+    <tdml:infoset>
+      <tdml:dfdlInfoset>
+        <tns:delimited>
+          <tns:foo>foo</tns:foo>
+          <tns:bar>bar</tns:bar>
+          <tns:baz>b,z</tns:baz>
+          <tns:qux>qux</tns:qux>
+        </tns:delimited>
+      </tdml:dfdlInfoset>
+    </tdml:infoset>
+
+  </tdml:parserTestCase>
+
 </tdml:testSuite>
diff --git a/daffodil-test/src/test/scala/org/apache/daffodil/section06/entities/TestEntities.scala b/daffodil-test/src/test/scala/org/apache/daffodil/section06/entities/TestEntities.scala
index 50678d3..d5de657 100644
--- a/daffodil-test/src/test/scala/org/apache/daffodil/section06/entities/TestEntities.scala
+++ b/daffodil-test/src/test/scala/org/apache/daffodil/section06/entities/TestEntities.scala
@@ -92,16 +92,23 @@ class TestEntities {
   @Test def test_whitespace_09() { runner_01.runOneTest("whitespace_09") }
   @Test def test_whitespace_10() { runner_01.runOneTest("whitespace_10") }
 
-  // JIRA DFDL-1475 - Broken currently - test was wrong. Should expect error. Test changed, but diagnostic is not acceptable.
-  // @Test def test_emptyStringEntityTermInExpression_01() { runner_01.runOneTest("emptyStringEntityTermInExpression_01") }
+  // DAFFODIL-1475
+  @Test def test_emptyStringEntityTermInExpression_01() { runner_01.runOneTest("emptyStringEntityTermInExpression_01") }
   @Test def test_emptyStringEntityTermInExpression_02() { runner_01.runOneTest("emptyStringEntityTermInExpression_02") }
+  @Test def test_emptyStringEntityTermInExpressionDelimited_01() { runner_01.runOneTest("emptyStringEntityTermInExpressionDelimited_01") }
+  @Test def test_emptyStringEntityTermInComplex_01() { runner_01.runOneTest("emptyStringEntityTermInComplex_01") }
+  @Test def test_emptyStringEntityTermInComplex_02() { runner_01.runOneTest("emptyStringEntityTermInComplex_02") }
+
+  @Test def test_emptyStringEntityInitiator_01() { runner_01.runOneTest("emptyStringEntityInitiator_01") }
+  @Test def test_emptyStringEntityInitiator_02() { runner_01.runOneTest("emptyStringEntityInitiator_02") }
+  @Test def test_emptyStringEntityInitiator_03() { runner_01.runOneTest("emptyStringEntityInitiator_03") }
 
   @Test def test_entity_fail_01() { runnerEntity.runOneTest("entity_fail_01") }
   @Test def test_entity_fail_02() { runnerEntity.runOneTest("entity_fail_02") }
 
-  // Regression - we used to just reject %ES; in terminators. Now we accept it, but it doesn't work
-  // right. JIRA DFDL-1477
-  // @Test def test_entity_fail_03() { runnerEntity.runOneTest("entity_fail_03") }
+  // DAFFODIL-1477
+  @Test def test_entity_fail_03a() { runnerEntity.runOneTest("entity_fail_03a") }
+  @Test def test_entity_fail_03b() { runnerEntity.runOneTest("entity_fail_03b") }
   @Test def test_entity_fail_04() { runnerEntity.runOneTest("entity_fail_04") }
 
   @Test def test_invalid_entity_01() { runnerInvalid.runOneTest("text_invalid_entity_name") }
diff --git a/daffodil-test/src/test/scala/org/apache/daffodil/section14/sequence_groups/TestSequenceGroups.scala b/daffodil-test/src/test/scala/org/apache/daffodil/section14/sequence_groups/TestSequenceGroups.scala
index 6616f7d..2d1de1e 100644
--- a/daffodil-test/src/test/scala/org/apache/daffodil/section14/sequence_groups/TestSequenceGroups.scala
+++ b/daffodil-test/src/test/scala/org/apache/daffodil/section14/sequence_groups/TestSequenceGroups.scala
@@ -91,4 +91,9 @@ class TestSequenceGroups {
 
   @Test def test_noDefaultSeqKind() { runner_02.runOneTest("noDefaultSeqKind") }
   @Test def test_sequenceWithComplexType() { runner_02.runOneTest("sequenceWithComplexType") }
+
+  // DAFFODIL-2171
+  @Test def test_delimiterScanning_01() { runner_01.runOneTest("delimiterScanning_01") }
+  @Test def test_delimiterScanning_02() { runner_01.runOneTest("delimiterScanning_02") }
+  //@Test def test_delimiterScanning_03() { runner_01.runOneTest("delimiterScanning_03") }
 }