You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@daffodil.apache.org by GitBox <gi...@apache.org> on 2020/01/07 15:26:27 UTC

[GitHub] [incubator-daffodil] stevedlawrence opened a new pull request #313: Refactor text numbers and related primitives/parsers

stevedlawrence opened a new pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313
 
 
   Note: This is PR started out as work for DAFFODIL-840
   (implement textStandardBase), but I quickly realized some refactoring
   needed to take place to make that implementation much neater. That refactoring
   ballooned into more refactoring, and turned this into a bunch of refactorings
   that are distinct but are related.
   
   For each text number parser, we allocated a helper which did things like
   check if a number was in a valid range, or convert a parsed number to
   the correct type. Many of the actions were actually static and aren't
   really specific to text numbers, but are more specific to the actual
   data type (e.g. int, float). This moves all that static information out
   of helpers and into NodeInfo Primitives.
   
   The information in the helper that isn't static is really just specific
   to the single text number parser, and so is just passed in directly to
   the parser. This also gets rid of the NumberFormatFactory and
   CacheDynamic stuff and replaces it with the more modern Evaluatable.
   
   This allows completely removes the helpers and even all the different
   text number primitives with type parameter, removing a lot of
   unnecessary duplication.
   
   This also led to the refactoring of other parts of code that do range
   checking, particularly DPath conversions. That removed a lot of
   duplicate range checking logic and instead uses the new range checker in
   NodeInfo. This also uncovered a handful of cases where we were too lax
   or incorrect in the DataValues that were being uses (e.g. a dpath
   function should return xs:unsignedByte but returned a JLong instead of
   JShort). These changes make conversion much more strict about which
   numeric types are expected to enforce correctness. This also avoids
   unnecessary conversions since we now know exactly which types things
   should be.
   
   Additionally, it was discovered that binary numbers do not enforce range
   checking, so that is now checked and a PE created if a binary number is
   too large for the type. That is the only functional change here.
   
   DAFFODIL-942

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r364004721
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/EvTextNumber.scala
 ##########
 @@ -51,6 +66,163 @@ class TextStandardExponentRepEv(expr: CompiledExpression[String], tci: DPathComp
   override lazy val runtimeDependencies = Vector()
 }
 
+class TextNumberFormatEv(
+  tci: DPathCompileInfo,
+  decimalSepEv: Maybe[TextStandardDecimalSeparatorEv],
+  groupingSepEv: Maybe[TextStandardGroupingSeparatorEv],
+  exponentRepEv: Maybe[TextStandardExponentRepEv],
+  infRep: Maybe[String],
+  nanRep: Maybe[String],
+  checkPolicy: TextNumberCheckPolicy,
+  textNumberPattern: String,
+  rounding: TextNumberRounding,
+  roundingMode: Maybe[TextNumberRoundingMode],
+  roundingIncrement: MaybeDouble,
+  zeroRepsRaw: List[String],
+  primType: PrimType)
+  extends Evaluatable[ThreadLocal[DecimalFormat]](tci)
+  with InfosetCachedEvaluatable[ThreadLocal[DecimalFormat]] {
 
 Review comment:
   Yeah, it's definitely feels strange.
   
   And yep, ThreadLocal does need to be Serializable but it's not. I found that if I just mixed Serializable in everything worked as expected. This is somewhere in this patch:
   ```scala
   val res = new ThreadLocal[DecimalFormat] with Serializable { ... }
   ```
   I think maybe the difference between the charset cache and this is there's a simple way to key on an already allcoated charset encoder/decoder--it's just the name of the charset. But with decimal format, there isn't a simple key--it takes a bunch of different parameters.
   
   That said, that sort of cache probably makes a ton of sense. Very few schemas are going to have more than a couple decimal formats, so having a unique one for every since text number parser is a pretty big waste of memory. I think that's a bit out of scope for this refactoring though (this is basically the same thing we had before, just using Evaluatables instead of CacheDynamic). I'll create a bug to think about how we can implement this to reduce the number of DecimalFormats allocated.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363808876
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/ZonedTextParsers.scala
 ##########
 @@ -47,70 +48,69 @@ case class ConvertZonedCombinatorParser(
   }
 }
 
-case class ConvertZonedNumberParser[S](
-  helper: ConvertTextNumberParserUnparserHelperBase[S],
+case class ConvertZonedNumberParser(
   opl: OverpunchLocation.Value,
-  nff: NumberFormatFactoryBase[S],
+  textNumberFormatEv: TextNumberFormatEv,
   zonedSignStyle: TextZonedSignStyle,
   override val context: ElementRuntimeData) extends TextPrimParser {
-  override lazy val runtimeDependencies = Vector()
 
-  override def toString = "to(xs:" + helper.xsdType + ")"
+  override lazy val runtimeDependencies = Vector(textNumberFormatEv)
+
+  private val primNumeric = context.optPrimType.get.asInstanceOf[NodeInfo.PrimType.PrimNumeric]
 
   def parse(start: PState): Unit = {
     val node: DISimple = start.simpleElement
     val str = node.dataValueAsString
 
     Assert.invariant(str != null) // worst case it should be empty string. But not null.
     if (str == "") {
-      PE(start, "Convert to %s (for xs:%s): Cannot parse number from empty string", helper.prettyType, helper.xsdType)
+      PE(start, "Unable to parse zoned %s from empty string", context.optPrimType.get.globalQName)
       return
     }
 
     var checkLength = str.length
     val numValue = {
-      val df = nff.getNumFormat(start)
+      val df = textNumberFormatEv.evaluate(start)
       val pos = new ParsePosition(0)
-      val num = try {
-        val decodedNum = DecimalUtils.zonedToNumber(str, zonedSignStyle, opl)
-        if (decodedNum(0) == '-')
-          checkLength = checkLength + 1
-        df.get.parse(decodedNum, pos)
+
+      val decodedNum = try {
+        DecimalUtils.zonedToNumber(str, zonedSignStyle, opl)
       } catch {
-        case s: scala.util.control.ControlThrowable => throw s
-        case u: UnsuppressableException => throw u
-        case e: Exception => {
 
 Review comment:
   These case never actually cought anything. I think they were copied from the text number parser, which also never caught anything and were removed in this PR. I suspect they could have happened in an older versions of ICU.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363807958
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/TextNumberParsers.scala
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.daffodil.processors.parsers
 
 Review comment:
   This file isn't actually entirely new. I just rename TextNumberPrimitives1.scala to a more appropriate TextNumberParsers. I guess GitHub can't detect that rename. The bulk of this is the same except for using the new evaluatable, using primNumeric for range checking/type conversion, and tweaks to error messages to make them more consistent.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363987218
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/EvTextNumber.scala
 ##########
 @@ -51,6 +66,163 @@ class TextStandardExponentRepEv(expr: CompiledExpression[String], tci: DPathComp
   override lazy val runtimeDependencies = Vector()
 }
 
+class TextNumberFormatEv(
+  tci: DPathCompileInfo,
+  decimalSepEv: Maybe[TextStandardDecimalSeparatorEv],
+  groupingSepEv: Maybe[TextStandardGroupingSeparatorEv],
+  exponentRepEv: Maybe[TextStandardExponentRepEv],
+  infRep: Maybe[String],
+  nanRep: Maybe[String],
+  checkPolicy: TextNumberCheckPolicy,
+  textNumberPattern: String,
+  rounding: TextNumberRounding,
+  roundingMode: Maybe[TextNumberRoundingMode],
+  roundingIncrement: MaybeDouble,
+  zeroRepsRaw: List[String],
+  primType: PrimType)
+  extends Evaluatable[ThreadLocal[DecimalFormat]](tci)
+  with InfosetCachedEvaluatable[ThreadLocal[DecimalFormat]] {
 
 Review comment:
   Doesn't that mean a new DecimalFormat will be allocated and initialized for every parse? That can be expensive. As I understand it, the purpose of the const member in Evaluatable is if something can be allocated and initialized at schema compile time then we do it then, avoiding that overhead during parse time. The consequene of this though, is that different threads now share that same allocated and initialized constant, which in the DecimalFormat case makes things not thread-safe.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363823678
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/EvTextNumber.scala
 ##########
 @@ -51,6 +66,163 @@ class TextStandardExponentRepEv(expr: CompiledExpression[String], tci: DPathComp
   override lazy val runtimeDependencies = Vector()
 }
 
+class TextNumberFormatEv(
+  tci: DPathCompileInfo,
+  decimalSepEv: Maybe[TextStandardDecimalSeparatorEv],
+  groupingSepEv: Maybe[TextStandardGroupingSeparatorEv],
+  exponentRepEv: Maybe[TextStandardExponentRepEv],
+  infRep: Maybe[String],
+  nanRep: Maybe[String],
+  checkPolicy: TextNumberCheckPolicy,
+  textNumberPattern: String,
+  rounding: TextNumberRounding,
+  roundingMode: Maybe[TextNumberRoundingMode],
+  roundingIncrement: MaybeDouble,
+  zeroRepsRaw: List[String],
+  primType: PrimType)
+  extends Evaluatable[ThreadLocal[DecimalFormat]](tci)
+  with InfosetCachedEvaluatable[ThreadLocal[DecimalFormat]] {
 
 Review comment:
   Since this is an evaluatable, this threadlocal can just be a member of the state now easily. Since states are per-thread anyway that is. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363828824
 
 

 ##########
 File path: daffodil-core/src/main/scala/org/apache/daffodil/grammar/primitives/PrimitivesZoned.scala
 ##########
 @@ -137,8 +119,8 @@ abstract class ConvertZonedNumberPrim[S](e: ElementBase)
         case TextNumberRounding.Pattern => (MaybeDouble.Nope, Nope)
       }
 
-    val nff = new NumberFormatFactoryStatic[S](
-      e.termRuntimeData, h,
+    val ev = new TextNumberFormatEv(
 
 Review comment:
   Yeah, i'll undo the renames so that the diffs make sense, and we can do that in a later commit. There are probably a bunch more than I didn't notice that need to be renamed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363830697
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/EvTextNumber.scala
 ##########
 @@ -51,6 +66,163 @@ class TextStandardExponentRepEv(expr: CompiledExpression[String], tci: DPathComp
   override lazy val runtimeDependencies = Vector()
 }
 
+class TextNumberFormatEv(
+  tci: DPathCompileInfo,
+  decimalSepEv: Maybe[TextStandardDecimalSeparatorEv],
+  groupingSepEv: Maybe[TextStandardGroupingSeparatorEv],
+  exponentRepEv: Maybe[TextStandardExponentRepEv],
+  infRep: Maybe[String],
+  nanRep: Maybe[String],
+  checkPolicy: TextNumberCheckPolicy,
+  textNumberPattern: String,
+  rounding: TextNumberRounding,
+  roundingMode: Maybe[TextNumberRoundingMode],
+  roundingIncrement: MaybeDouble,
+  zeroRepsRaw: List[String],
+  primType: PrimType)
+  extends Evaluatable[ThreadLocal[DecimalFormat]](tci)
+  with InfosetCachedEvaluatable[ThreadLocal[DecimalFormat]] {
 
 Review comment:
   I thought if the Evaluatable is constant (which is probably the common case here since text number properties tend to not use expressions) then the result would be be computed only once at compile time and shared among threads? Or maybe I'm misunderstanding what you're suggesting?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363822061
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/dpath/NodeInfo.scala
 ##########
 @@ -439,8 +444,61 @@ object NodeInfo extends Enum {
       }
     }
 
+    trait PrimNumeric { self: Numeric.Kind =>
+      def isValidRange(n: Number): Boolean
+      def fromNumber(n: Number): DataValueNumber
+    }
+
+    // this should only be used for integer primitives that can fit inside a
+    // long (e.g. long, unsignedInt). Primirives larger than that should
 
 Review comment:
   Typo primirives

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363999872
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/EvTextNumber.scala
 ##########
 @@ -51,6 +66,163 @@ class TextStandardExponentRepEv(expr: CompiledExpression[String], tci: DPathComp
   override lazy val runtimeDependencies = Vector()
 }
 
+class TextNumberFormatEv(
+  tci: DPathCompileInfo,
+  decimalSepEv: Maybe[TextStandardDecimalSeparatorEv],
+  groupingSepEv: Maybe[TextStandardGroupingSeparatorEv],
+  exponentRepEv: Maybe[TextStandardExponentRepEv],
+  infRep: Maybe[String],
+  nanRep: Maybe[String],
+  checkPolicy: TextNumberCheckPolicy,
+  textNumberPattern: String,
+  rounding: TextNumberRounding,
+  roundingMode: Maybe[TextNumberRoundingMode],
+  roundingIncrement: MaybeDouble,
+  zeroRepsRaw: List[String],
+  primType: PrimType)
+  extends Evaluatable[ThreadLocal[DecimalFormat]](tci)
+  with InfosetCachedEvaluatable[ThreadLocal[DecimalFormat]] {
 
 Review comment:
   Ah. I was worried about the case where it's non-constant. That's not the problem. 
   
   I think if you have an evaluatable, that if it produces a constant value, but..... that constant is actually mutable non-thread-safe instance, ....it's a bit strange.
   
   But your techinque works fine for this, and ThreadLocal.get is very tight code that doesn't synchronize or anything,  so this is fine. 
   
   Doesn't ThreadLocal have to be Serializable in this case though? I don't think it is from looking at Javadoc,  but how could your code be working if it isn't? It's trivial for it to be serializable though, so maybe it is, and just isn't mentioned in javadoc?
   
   There is another analogous place which is the way Charset Encoder/Decoder work. Those are stateful non-thread safe, but potentially if encoding is known they are these quasi-constants in that they are needed to be per-thread instances. There's a CharsetEv, but not EncoderEv nor DecoderEv. Instead there's a cache for these maintained in the State. There's a bunch of other stuff going on there to deal with fast detection that since the charset didn't change the Encoder/Decoder can't change, but this techique using ThreadLocal still may be a simplifier as well there. 
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
stevedlawrence commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363845920
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/EvTextNumber.scala
 ##########
 @@ -51,6 +66,163 @@ class TextStandardExponentRepEv(expr: CompiledExpression[String], tci: DPathComp
   override lazy val runtimeDependencies = Vector()
 }
 
+class TextNumberFormatEv(
+  tci: DPathCompileInfo,
+  decimalSepEv: Maybe[TextStandardDecimalSeparatorEv],
+  groupingSepEv: Maybe[TextStandardGroupingSeparatorEv],
+  exponentRepEv: Maybe[TextStandardExponentRepEv],
+  infRep: Maybe[String],
+  nanRep: Maybe[String],
+  checkPolicy: TextNumberCheckPolicy,
+  textNumberPattern: String,
+  rounding: TextNumberRounding,
+  roundingMode: Maybe[TextNumberRoundingMode],
+  roundingIncrement: MaybeDouble,
+  zeroRepsRaw: List[String],
+  primType: PrimType)
+  extends Evaluatable[ThreadLocal[DecimalFormat]](tci)
+  with InfosetCachedEvaluatable[ThreadLocal[DecimalFormat]] {
 
 Review comment:
   My understanding is that there is single Evaluatable instance (e.g. TextNumberFormatEv) that is passed to a parser, and this Evaluatable is shared among threads. There is a "maybeConst" member variable in the Evaluatable that stores the compile time result of the Evaluatable if it is constant. In that case, that constant value will be shared between different threads when evaluate(state) is called. The state is never used when it's constant.
   
   In this particular case, DecimalFormat isn't thread-safe, so we need to have a unique instance per-thread, so the ThreadLocal is used for that. If the Evaluatable isn't constant, then we don't technically' need the ThreadLocal, since we'll get a new DecimalFormat everytime time  evaluate(state) is called, but I'm not sure there's a good way to handle a ThreadLocal[DecimalFormat] in once case or a DecimalFormat in another.
   
   One option would be to never cache the result and always allocate a new DecimalFormat everytime evaluate is called, but the creation of DecimalFormat is probably pretty slow since it needs to parse text number patterns and do all kinds of other stuff.
   
   Perhaps we really need an EvaluatableNonThreadSafe, which uses a ThreadLocal behind the scenes for when it's a constant?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] stevedlawrence merged pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
stevedlawrence merged pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363819897
 
 

 ##########
 File path: daffodil-core/src/main/scala/org/apache/daffodil/grammar/primitives/PrimitivesZoned.scala
 ##########
 @@ -137,8 +119,8 @@ abstract class ConvertZonedNumberPrim[S](e: ElementBase)
         case TextNumberRounding.Pattern => (MaybeDouble.Nope, Nope)
       }
 
-    val nff = new NumberFormatFactoryStatic[S](
-      e.termRuntimeData, h,
+    val ev = new TextNumberFormatEv(
 
 Review comment:
   This change set is going to fix numerous outstanding cleanup tickets. 
   
   The thought that crosses my mind is whether you want to follow this change set up with a separate commit that just does file-reorganization of the resulting code. I am often tempted to do that but then don't because it makes diffs for review problematic, but if you do it as two separate PRs, one of which is only the file reorg, that's less an issue. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363838338
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/EvTextNumber.scala
 ##########
 @@ -51,6 +66,163 @@ class TextStandardExponentRepEv(expr: CompiledExpression[String], tci: DPathComp
   override lazy val runtimeDependencies = Vector()
 }
 
+class TextNumberFormatEv(
+  tci: DPathCompileInfo,
+  decimalSepEv: Maybe[TextStandardDecimalSeparatorEv],
+  groupingSepEv: Maybe[TextStandardGroupingSeparatorEv],
+  exponentRepEv: Maybe[TextStandardExponentRepEv],
+  infRep: Maybe[String],
+  nanRep: Maybe[String],
+  checkPolicy: TextNumberCheckPolicy,
+  textNumberPattern: String,
+  rounding: TextNumberRounding,
+  roundingMode: Maybe[TextNumberRoundingMode],
+  roundingIncrement: MaybeDouble,
+  zeroRepsRaw: List[String],
+  primType: PrimType)
+  extends Evaluatable[ThreadLocal[DecimalFormat]](tci)
+  with InfosetCachedEvaluatable[ThreadLocal[DecimalFormat]] {
 
 Review comment:
   I just saw evaluatable parameterized by thread-local type, and that seems incongruous to me. A thread-local is really a hack that uses the current thread ID and a hash table to find a thread-specific instance. We don't need to use that mechanism since the state is a thread-specific object. 
   
   So a thread-local and an evaluatable should never go together.
   
   Even if everything folds away to a constant you still sometimes need the DecimalFormat object at runtime (I think.... ) 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363839558
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/EvTextNumber.scala
 ##########
 @@ -51,6 +66,163 @@ class TextStandardExponentRepEv(expr: CompiledExpression[String], tci: DPathComp
   override lazy val runtimeDependencies = Vector()
 }
 
+class TextNumberFormatEv(
+  tci: DPathCompileInfo,
+  decimalSepEv: Maybe[TextStandardDecimalSeparatorEv],
+  groupingSepEv: Maybe[TextStandardGroupingSeparatorEv],
+  exponentRepEv: Maybe[TextStandardExponentRepEv],
+  infRep: Maybe[String],
+  nanRep: Maybe[String],
+  checkPolicy: TextNumberCheckPolicy,
+  textNumberPattern: String,
+  rounding: TextNumberRounding,
+  roundingMode: Maybe[TextNumberRoundingMode],
+  roundingIncrement: MaybeDouble,
+  zeroRepsRaw: List[String],
+  primType: PrimType)
+  extends Evaluatable[ThreadLocal[DecimalFormat]](tci)
+  with InfosetCachedEvaluatable[ThreadLocal[DecimalFormat]] {
+
+  override lazy val runtimeDependencies = (decimalSepEv.toList ++ groupingSepEv.toList ++ exponentRepEv.toList).toVector
+
+  private val isInt = primType match {
+    case PrimType.Double | PrimType.Float | PrimType.Decimal => false
+    case _ => true
+  }
+
+  private def checkUnique(
+    decimalSep: MaybeChar,
+    groupingSep: MaybeChar,
+    exponentRep: Maybe[String]): Unit = {
+
+    val mm = new mutable.HashMap[String, mutable.Set[String]] with mutable.MultiMap[String, String]
+    if (decimalSep.isDefined) mm.addBinding(decimalSep.get.toString, "textStandardDecimalSeparator")
+    if (groupingSep.isDefined) mm.addBinding(groupingSep.get.toString, "textStandardGroupingSeparator")
+    if (exponentRep.isDefined) mm.addBinding(exponentRep.get, "textStandardExponentRep")
+    if (infRep.isDefined) mm.addBinding(infRep.get, "textStandardInfinityRep")
+    if (nanRep.isDefined) mm.addBinding(nanRep.get, "textStandardNaNRep")
+    zeroRepsRaw.foreach { zr => mm.addBinding(zr, "textStandardZeroRep") }
+
+    val dupes = mm.filter { case (k, s) => s.size > 1 }
+    val dupeStrings = dupes.map {
+      case (k, s) =>
+        "Non-distinct property '%s' found in: %s".format(k, s.mkString(", "))
+    }
+    tci.schemaDefinitionUnless(dupeStrings.size == 0, dupeStrings.mkString("\n"))
+  }
+
+  private def generateNumFormat(
+    decimalSep: MaybeChar,
+    groupingSep: MaybeChar,
+    exponentRep: Maybe[String]): DecimalFormat = {
+
+    val dfs = new DecimalFormatSymbols()
+
+    if (decimalSep.isDefined) {
+      dfs.setDecimalSeparator(decimalSep.get)
+    }
+
+    if (groupingSep.isDefined) {
+      dfs.setGroupingSeparator(groupingSep.get)
+    }
+
+    // TODO: this is allowed to be case insenstive, ICU doesn't support that
+    if (exponentRep.isDefined) {
+      dfs.setExponentSeparator(exponentRep.get)
+    }
+
+    if (infRep.isDefined) {
+      // TODO: this is allowed to be case insensitive, ICU doesn't support that
+      dfs.setInfinity(infRep.get)
+    }
+
+    if (nanRep.isDefined) {
+      // TODO: this is allowed to be case insensitive, ICU doesn't support that
+      dfs.setNaN(nanRep.get)
+    }
+
+    val df = new DecimalFormat(textNumberPattern, dfs)
+
+    val cp = checkPolicy match {
+      case TextNumberCheckPolicy.Strict => true
+      case TextNumberCheckPolicy.Lax => false
+    }
+    df.setParseStrict(cp)
+
+    rounding match {
+      case TextNumberRounding.Pattern => {
+        df.setRoundingMode(RoundingMode.HALF_EVEN.ordinal())
+      }
+      case TextNumberRounding.Explicit => {
+        val rm = roundingMode.get match {
+          case TextNumberRoundingMode.RoundCeiling => RoundingMode.CEILING
+          case TextNumberRoundingMode.RoundFloor => RoundingMode.FLOOR
+          case TextNumberRoundingMode.RoundDown => RoundingMode.DOWN
+          case TextNumberRoundingMode.RoundUp => RoundingMode.UP
+          case TextNumberRoundingMode.RoundHalfEven => RoundingMode.HALF_EVEN
+          case TextNumberRoundingMode.RoundHalfDown => RoundingMode.HALF_DOWN
+          case TextNumberRoundingMode.RoundHalfUp => RoundingMode.HALF_UP
+          case TextNumberRoundingMode.RoundUnnecessary => RoundingMode.UNNECESSARY
+        }
+        df.setRoundingMode(rm.ordinal())
+        df.setRoundingIncrement(roundingIncrement.get)
+      }
+    }
+
+    if (isInt) {
+      df.setMaximumFractionDigits(0)
+      df.setDecimalSeparatorAlwaysShown(false)
+      df.setParseIntegerOnly(true)
+    }
+
+    df
+  }
+
+  override protected def compute(state: ParseOrUnparseState): ThreadLocal[DecimalFormat] = {
+
+    val decimalSepList = if (decimalSepEv.isDefined) {
+      val seps = decimalSepEv.get.evaluate(state)
+      if (seps.length > 1) {
+        // TODO: ICU only supports a single decimal separator
+        tci.notYetImplemented("More than one textStandardDecimalSeparator")
+      }
+      MaybeChar(seps.head(0))
+    } else {
+      MaybeChar.Nope
+    }
+
+    val groupingSep = if (groupingSepEv.isDefined) {
+      MaybeChar(groupingSepEv.get.evaluate(state)(0))
+    } else {
+      MaybeChar.Nope
+    }
+
+    val exponentRep = if (exponentRepEv.isDefined) {
+      One(exponentRepEv.get.evaluate(state))
+    } else {
+      Nope
+    }
+
+    checkUnique(
+      decimalSepList,
+      groupingSep,
+      exponentRep)
+
+    val numFormat = new ThreadLocal[DecimalFormat] with Serializable {
 
 Review comment:
   This right here is entirely weird to me. Why would you allocate a thread-local here, when state is passed to this method. 
   
   Can't we just access a method on state here that constructs unique decimal formats for this thread.  

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363825989
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/parsers/TextNumberParsers.scala
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.daffodil.processors.parsers
 
 Review comment:
   I have made this mistake repeatedly as well - renaming things, then finding that the PR can't diff things properly. 
   
   I now advocate renaming files as a last separate commit, or even a separate PR. It's hard to resist when you are cleaning things up though. Particularly if you expect git to detect the rename, which it sometimes does. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-daffodil] mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers

Posted by GitBox <gi...@apache.org>.
mbeckerle commented on a change in pull request #313: Refactor text numbers and related primitives/parsers
URL: https://github.com/apache/incubator-daffodil/pull/313#discussion_r363984882
 
 

 ##########
 File path: daffodil-runtime1/src/main/scala/org/apache/daffodil/processors/EvTextNumber.scala
 ##########
 @@ -51,6 +66,163 @@ class TextStandardExponentRepEv(expr: CompiledExpression[String], tci: DPathComp
   override lazy val runtimeDependencies = Vector()
 }
 
+class TextNumberFormatEv(
+  tci: DPathCompileInfo,
+  decimalSepEv: Maybe[TextStandardDecimalSeparatorEv],
+  groupingSepEv: Maybe[TextStandardGroupingSeparatorEv],
+  exponentRepEv: Maybe[TextStandardExponentRepEv],
+  infRep: Maybe[String],
+  nanRep: Maybe[String],
+  checkPolicy: TextNumberCheckPolicy,
+  textNumberPattern: String,
+  rounding: TextNumberRounding,
+  roundingMode: Maybe[TextNumberRoundingMode],
+  roundingIncrement: MaybeDouble,
+  zeroRepsRaw: List[String],
+  primType: PrimType)
+  extends Evaluatable[ThreadLocal[DecimalFormat]](tci)
+  with InfosetCachedEvaluatable[ThreadLocal[DecimalFormat]] {
 
 Review comment:
   Pretty sure it's way simpler than this.
   
   Here's the key observation: The infoset is part of the state. Caching on the infoset is a private thread-safe cache. The Evaluatable is taking care of that for you.
   
   So I believe you can just always allocate a DecimalFormat as the result of the compute method. 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services