You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/12/29 12:54:59 UTC

[GitHub] [spark] beliefer opened a new pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

beliefer opened a new pull request #35060:
URL: https://github.com/apache/spark/pull/35060


   ### What changes were proposed in this pull request?
   Many database support the function `to_number` to convert a string to number. 
   The implement of `to_number` has many different between `Postgresql` ,`Oracle` and `Phoenix`.
   So, this PR follows the implement of `to_number` in `Oracle` that give a strict parameter verification.
   So, this PR follows the implement of `to_number` in `Phoenix` that uses BigDecimal.
   
   This PR support the patterns for numeric formatting as follows:
   
   Pattern | Description
   -- | --
   9 | Value with the specified number of digits
   0 | Value with leading zeros
   . (period) | Decimal point
   , (comma) | Group (thousand) separator
   S | Sign anchored to number (uses locale)
   $ | a value with a leading dollar sign
   D | Decimal point (uses locale)
   G | Group separator (uses locale)
   
   
   
   There are some mainstream database support the syntax.
   **PostgreSQL:**
   https://www.postgresql.org/docs/12/functions-formatting.html
   
   **Oracle:**
   https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/TO_NUMBER.html#GUID-D4807212-AFD7-48A7-9AED-BEC3E8809866
   
   **Vertica**
   https://www.vertica.com/docs/10.0.x/HTML/Content/Authoring/SQLReferenceManual/Functions/Formatting/TO_NUMBER.htm?tocpath=SQL%20Reference%20Manual%7CSQL%20Functions%7CFormatting%20Functions%7C_____7
   
   **Redshift**
   https://docs.aws.amazon.com/redshift/latest/dg/r_TO_NUMBER.html
   
   **DB2**
   https://www.ibm.com/support/knowledgecenter/SSGU8G_14.1.0/com.ibm.sqls.doc/ids_sqs_1544.htm
   
   **Teradata**
   https://docs.teradata.com/r/kmuOwjp1zEYg98JsB8fu_A/TH2cDXBn6tala29S536nqg
   
   **Snowflake:**
   https://docs.snowflake.net/manuals/sql-reference/functions/to_decimal.html
   
   **Exasol**
   https://docs.exasol.com/sql_references/functions/alphabeticallistfunctions/to_number.htm#TO_NUMBER
   
   **Phoenix**
   http://phoenix.incubator.apache.org/language/functions.html#to_number
   
   **Singlestore**
   https://docs.singlestore.com/v7.3/reference/sql-reference/numeric-functions/to-number/
   
   **Intersystems**
   https://docs.intersystems.com/latest/csp/docbook/DocBook.UI.Page.cls?KEY=RSQL_TONUMBER
   
   The syntax like:
   > select to_number('12,454.8-', '99G999D9S');
   -12454.8
   
   ### Why are the changes needed?
   `to_number` is very useful for formatted currency to number conversion.
   
   
   ### Does this PR introduce any user-facing change?
   Yes. New feature.
   
   
   ### How was this patch tested?
   New tests
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786105657



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatter(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {
+    val decimalFormat = new DecimalFormat(transformedFormat)
+    decimalFormat.setParseBigDecimal(true)
+    decimalFormat
+  }
+
+  private val precision = normalizedNumberFormat.filterNot(isSign).length
+
+  private val scale = {
+    val formatSplits = normalizedNumberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
+    }
+  }
+
+  def parsedDecimalType: DecimalType = DecimalType(precision, scale)

Review comment:
       OK since we still calculate precision and scale in two passes, I don't think we need this anymore and can just create `DecimalType` in the `dataType` implementation




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779990808



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0999|
+|**.**|decimal point (only allowed once)|99.99|
+|**D**|decimal point (only allowed once)|99D99|
+|**,**|group (thousands) separator|9,999|
+|**G**|group (thousands) separator|9G999|
+|**-**|sign anchored to number|-9999|

Review comment:
       PostgreSQL throws `ERROR: cannot use "S" twice`
   Oracle throws `ORA-01481: 无效的数字格式模型`.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r777834924



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)
+         '.':  decimal point (only allowed once)
+         ',':  group (thousands) separator
+         'S':  sign anchored to number (uses locale)
+         'D':  decimal point (uses locale)

Review comment:
       the Spark doc should match Spark implementation




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r777834433



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position
+         '0':  digit position too

Review comment:
       `'0' or '9': digit position`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781186203



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -136,21 +164,32 @@ object NumberUtils {
       inputSplits(1).filterNot(isSign).length > scale) {
       throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
     }
+
+    val transformedFormat = transform(normalizedNumberFormat)
+    val numberFormatInstance = NumberFormat.getNumberInstance(Locale.ROOT)
+    assert(numberFormatInstance.isInstanceOf[DecimalFormat])

Review comment:
       why can't we just do `new DecimalFormat()`?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783662327



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,174 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("99")))
+    }
+
+    Seq(
+      ("454", "999") -> Decimal(454),
+      ("054", "999") -> Decimal(54),
+      ("54", "999") -> Decimal(54),
+      ("404", "999") -> Decimal(404),
+      ("450", "999") -> Decimal(450),
+      ("454", "9999") -> Decimal(454),
+      ("054", "9999") -> Decimal(54),
+      ("404", "9999") -> Decimal(404),
+      ("450", "9999") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("0")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("00")))
+    }
+
+    Seq(
+      ("454", "000") -> Decimal(454),
+      ("054", "000") -> Decimal(54),
+      ("54", "000") -> Decimal(54),
+      ("404", "000") -> Decimal(404),
+      ("450", "000") -> Decimal(450),
+      ("454", "0000") -> Decimal(454),
+      ("054", "0000") -> Decimal(54),
+      ("404", "0000") -> Decimal(404),
+      ("450", "0000") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    // Test '.' and 'D'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.2"), Literal("999")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.23"), Literal("999.9")))
+    }
+
+    Seq(
+      ("454.2", "999.9") -> Decimal(454.2),

Review comment:
       For 2, It seems worth do the compare when calling format. It's necessary when calling parse.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784510828



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.sql.AnalysisException
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatter(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private lazy val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {
+    val decimalFormat = new DecimalFormat(transformedFormat)
+    decimalFormat.setParseBigDecimal(true)
+    decimalFormat
+  }
+
+  private lazy val precision = normalizedNumberFormat.filterNot(isSign).length
+
+  private lazy val scale = {
+    val formatSplits = normalizedNumberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
+    }
+  }
+
+  def parsedDecimalType: DecimalType = DecimalType(precision, scale)
+
+  /**
+   * DecimalFormat provides '#' and '0' as placeholder of digit, ',' as grouping separator,
+   * '.' as decimal separator, '-' as minus, '$' as dollar, but not '9', 'G', 'D', 'S'. So we need
+   * replace them show below:
+   * 1. '9' -> '#'
+   * 2. 'G' -> ','
+   * 3. 'D' -> '.'
+   * 4. 'S' -> '-'
+   *
+   * Note: When calling format, we must preserve the digits after decimal point, so the digits
+   * after decimal point should be replaced as '0'. For example: '999.9' will be normalized as
+   * '###.0' and '999.99' will be normalized as '###.00', so if the input is 454, the format
+   * output will be 454.0 and 454.00 respectively.
+   *
+   * @param format number format string
+   * @return normalized number format string
+   */
+  private def normalize(format: String): String = {
+    var notFindDecimalPoint = true
+    val normalizedFormat = format.toUpperCase(Locale.ROOT).map {
+      case NINE_DIGIT if notFindDecimalPoint => POUND_SIGN
+      case NINE_DIGIT if !notFindDecimalPoint => ZERO_DIGIT
+      case COMMA_LETTER => COMMA_SIGN
+      case POINT_LETTER | POINT_SIGN =>
+        notFindDecimalPoint = false
+        POINT_SIGN
+      case MINUS_LETTER => MINUS_SIGN
+      case other => other
+    }
+    // If the comma is at the beginning or end of number format, then DecimalFormat will be
+    // invalid. For example, "##,###," or ",###,###" for DecimalFormat is invalid, so we must use
+    // "##,###" or "###,###".
+    normalizedFormat.stripPrefix(COMMA_SIGN_STRING).stripSuffix(COMMA_SIGN_STRING)
+  }
+
+  private def isSign(c: Char): Boolean = {
+    SIGN_SET.contains(c)
+  }
+
+  private def transform(format: String): String = {
+    if (format.contains(MINUS_SIGN)) {
+      // For example: '#.######' represents a positive number,
+      // but '#.######;#.######-' represents a negative number.
+      val positiveFormatString = format.replaceAll("-", "")
+      s"$positiveFormatString;$format"
+    } else {
+      format
+    }
+  }
+
+  def check(): TypeCheckResult = {
+    def invalidSignPosition(c: Char): Boolean = {
+      val signIndex = normalizedNumberFormat.indexOf(c)
+      signIndex > 0 && signIndex < normalizedNumberFormat.length - 1
+    }
+
+    def multipleSignInNumberFormatError(message: String): String = {
+      s"Multiple $message in '$originNumberFormat'"
+    }
+
+    def nonFistOrLastCharInNumberFormatError(message: String): String = {
+      s"$message must be the first or last char in '$originNumberFormat'"
+    }
+
+    if (normalizedNumberFormat.length == 0) {
+      TypeCheckResult.TypeCheckFailure("Number format cannot be empty")
+    } else if (normalizedNumberFormat.count(_ == POINT_SIGN) > 1) {
+      TypeCheckResult.TypeCheckFailure(
+        multipleSignInNumberFormatError(s"'$POINT_LETTER' or '$POINT_SIGN'"))
+    } else if (normalizedNumberFormat.count(_ == MINUS_SIGN) > 1) {
+      TypeCheckResult.TypeCheckFailure(
+        multipleSignInNumberFormatError(s"'$MINUS_LETTER' or '$MINUS_SIGN'"))
+    } else if (normalizedNumberFormat.count(_ == DOLLAR_SIGN) > 1) {
+      TypeCheckResult.TypeCheckFailure(multipleSignInNumberFormatError(s"'$DOLLAR_SIGN'"))
+    } else if (invalidSignPosition(MINUS_SIGN)) {
+      TypeCheckResult.TypeCheckFailure(
+        nonFistOrLastCharInNumberFormatError(s"'$MINUS_LETTER' or '$MINUS_SIGN'"))
+    } else if (invalidSignPosition(DOLLAR_SIGN)) {

Review comment:
       PostgreSQL support, Oracle not.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776909773



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)
+         '.':  decimal point (only allowed once)
+         ',':  group (thousands) separator
+         'S':  sign anchored to number (uses locale)
+         'D':  decimal point (uses locale)

Review comment:
       This comment references PostgreSQL and Oracle. But other database does not mention locale.
   I guess `NumberFormat.getNumberInstance(Locale.ROOT)` is enough.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r780091667



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,44 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0000|

Review comment:
       `Position for a digit; When formatting, it adds leading/trailing 0.`

##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,44 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0000|

Review comment:
       similar for `9`
   ```
   Position for a digit; When formatting, it adds a leading/trailing blank space.
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781705914



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,172 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("99")))
+    }
+
+    Seq(
+      ("454", "999") -> Decimal(454),
+      ("054", "999") -> Decimal(54),

Review comment:
       Return `Decimal(54)`. Let's update tests.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r787792500



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,158 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+
+    Seq("454", "054", "54", "450").foreach { input =>
+      val invalidFormat1 = 0.until(input.length - 1).map(_ => '0').mkString
+      val invalidFormat2 = 0.until(input.length - 2).map(_ => '0').mkString
+      val invalidFormat3 = 0.until(input.length - 1).map(_ => '9').mkString
+      val invalidFormat4 = 0.until(input.length - 2).map(_ => '9').mkString
+      Seq(invalidFormat1, invalidFormat2, invalidFormat3, invalidFormat4)
+        .filter(_.nonEmpty).foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal(input), Literal(format)),
+          s"The input string '$input' does not match the given number format: '$format'")
+      }
+
+      val format1 = 0.until(input.length).map(_ => '0').mkString
+      val format2 = 0.until(input.length).map(_ => '9').mkString
+      val format3 = 0.until(input.length).map(i => i % 2 * 9).mkString
+      val format4 = 0.until(input.length + 1).map(_ => '0').mkString
+      val format5 = 0.until(input.length + 1).map(_ => '9').mkString
+      val format6 = 0.until(input.length + 1).map(i => i % 2 * 9).mkString
+      Seq(format1, format2, format3, format4, format5, format6).foreach { format =>
+        checkEvaluation(ToNumber(Literal(input), Literal(format)), Decimal(input))
+      }
+    }
+
+    // Test '.' and 'D'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.2"), Literal("999")),
+      "The input string '454.2' does not match the given number format: '999'")
+    Seq("999.9", "000.0", "99.99", "00.00", "0000.0", "9999.9", "00.000", "99.999")
+      .foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454.23"), Literal(format)),
+          s"The input string '454.23' does not match the given number format: '$format'")
+        val format2 = format.replace('.', 'D')
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454.23"), Literal(format2)),
+          s"The input string '454.23' does not match the given number format: '$format2'")
+    }
+
+    Seq(
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),
+      ("454.23", "000.00") -> Decimal(454.23),
+      ("454.23", "000D00") -> Decimal(454.23),
+      ("454.2", "000.00") -> Decimal(454.2),
+      ("454.2", "000D00") -> Decimal(454.2),
+      ("454.0", "000.0") -> Decimal(454),
+      ("454.0", "000D0") -> Decimal(454),
+      ("454.00", "000.00") -> Decimal(454),
+      ("454.00", "000D00") -> Decimal(454),
+      (".4542", ".0000") -> Decimal(0.4542),
+      (".4542", "D0000") -> Decimal(0.4542),
+      ("4542.", "0000.") -> Decimal(4542),
+      ("4542.", "0000D") -> Decimal(4542)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    Seq("999.9.9", "999D9D9", "999.9D9", "999D9.9").foreach { str =>
+      ToNumber(Literal("454.3.2"), Literal(str)).checkInputDataTypes() match {
+        case TypeCheckResult.TypeCheckFailure(msg) =>
+          assert(msg.contains(s"At most one 'D' or '.' is allowed in the number format: '$str'"))
+      }
+    }
+
+    // Test ',' and 'G'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("123,456"), Literal("9G9")),
+      "The input string '123,456' does not match the given number format: '9G9'")
+
+    Seq(
+      ("12,454", "99,999") -> Decimal(12454),
+      ("12,454", "00,000") -> Decimal(12454),
+      ("12,454,367", "99,999,999") -> Decimal(12454367),
+      ("12,454,367", "00,000,000") -> Decimal(12454367),
+      ("12,454,", "99,999,") -> Decimal(12454),
+      ("12,454,", "00,000,") -> Decimal(12454),
+      (",454,367", ",999,999") -> Decimal(454367),
+      (",454,367", ",000,000") -> Decimal(454367),
+      (",454,367", "999,999") -> Decimal(454367),
+      (",454,367", "000,000") -> Decimal(454367)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+      val format2 = format.replace(',', 'G')
+      checkEvaluation(ToNumber(Literal(str), Literal(format2)), expected)
+      val format3 = s"${format}0"
+      checkEvaluation(ToNumber(Literal(str), Literal(format3)), expected)
+      val format4 = s"0${format}9"
+      checkEvaluation(ToNumber(Literal(str), Literal(format4)), expected)
+    }
+
+    // Test '$'
+    Seq(
+      ("$78.12", "$99.99") -> Decimal(78.12),
+      ("$78.12", "$00.00") -> Decimal(78.12),
+      ("78.12$", "99.99$") -> Decimal(78.12),
+      ("78.12$", "00.00$") -> Decimal(78.12)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    ToNumber(Literal("$78$.12"), Literal("$99$.99")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("At most one '$' is allowed in the number format: '$99$.99'"))
+    }
+    ToNumber(Literal("78$.12"), Literal("99$.99")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("'$' must be the first or last char in the number format: '99$.99'"))
+    }
+
+    // Test '-' and 'S'
+    Seq(
+      ("454-", "999-") -> Decimal(-454),
+      ("454-", "999S") -> Decimal(-454),
+      ("-454", "-999") -> Decimal(-454),
+      ("-454", "S999") -> Decimal(-454),
+      ("454-", "000-") -> Decimal(-454),
+      ("454-", "000S") -> Decimal(-454),
+      ("-454", "-000") -> Decimal(-454),
+      ("-454", "S000") -> Decimal(-454),

Review comment:
       ditto, replace `9` with `0`, `-` with `S` in the loop body.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786436785



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatter(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {
+    val decimalFormat = new DecimalFormat(transformedFormat)
+    decimalFormat.setParseBigDecimal(true)
+    decimalFormat
+  }
+
+  private val precision = normalizedNumberFormat.filterNot(isSign).length
+
+  private val scale = {
+    val formatSplits = normalizedNumberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
+    }
+  }
+
+  def parsedDecimalType: DecimalType = DecimalType(precision, scale)

Review comment:
       `precision` and `scale` is used in other place.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786115746



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,179 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+
+    Seq("454", "054", "54", "450").foreach { input =>
+      val invalidFormat1 = 0.until(input.length - 1).map(_ => '0').mkString
+      val invalidFormat2 = 0.until(input.length - 2).map(_ => '0').mkString
+      val invalidFormat3 = 0.until(input.length - 1).map(_ => '9').mkString
+      val invalidFormat4 = 0.until(input.length - 2).map(_ => '9').mkString
+      Seq(invalidFormat1, invalidFormat2, invalidFormat3, invalidFormat4)
+        .filter(_.nonEmpty).foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454"), Literal(format)), s"Format '$format' used for" +
+            " parsing string to number or formatting number to string is invalid")
+      }
+
+      val format1 = 0.until(input.length).map(_ => '0').mkString
+      val format2 = 0.until(input.length).map(_ => '9').mkString
+      val format3 = 0.until(input.length + 1).map(_ => '0').mkString
+      val format4 = 0.until(input.length + 1).map(_ => '9').mkString
+      val format5 = 0.until(input.length + 2).map(_ => '0').mkString

Review comment:
       Can we test the mixed case as I proposed before? `val format3 = 0.until(input.length + 2).map(i => i % 2).mkString`
   
   It's usually not very useful to go deeper into one factor in the test. To test format string that is longer than the value, the `+ 1` case should be sufficient, we don't need to test `+ 2`. Ditto to the `- 2` case.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784012315



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,174 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("99")))
+    }
+
+    Seq(
+      ("454", "999") -> Decimal(454),
+      ("054", "999") -> Decimal(54),
+      ("54", "999") -> Decimal(54),
+      ("404", "999") -> Decimal(404),
+      ("450", "999") -> Decimal(450),
+      ("454", "9999") -> Decimal(454),
+      ("054", "9999") -> Decimal(54),
+      ("404", "9999") -> Decimal(404),
+      ("450", "9999") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("0")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("00")))
+    }
+
+    Seq(
+      ("454", "000") -> Decimal(454),
+      ("054", "000") -> Decimal(54),
+      ("54", "000") -> Decimal(54),
+      ("404", "000") -> Decimal(404),
+      ("450", "000") -> Decimal(450),
+      ("454", "0000") -> Decimal(454),
+      ("054", "0000") -> Decimal(54),
+      ("404", "0000") -> Decimal(404),
+      ("450", "0000") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    // Test '.' and 'D'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.2"), Literal("999")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.23"), Literal("999.9")))
+    }
+
+    Seq(
+      ("454.2", "999.9") -> Decimal(454.2),
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "999D9") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),
+      ("454.23", "999.99") -> Decimal(454.23),
+      ("454.23", "000.00") -> Decimal(454.23),
+      ("454.23", "999D99") -> Decimal(454.23),
+      ("454.23", "000D00") -> Decimal(454.23),
+      ("454.0", "999.9") -> Decimal(454),
+      ("454.0", "000.0") -> Decimal(454),
+      ("454.0", "999D9") -> Decimal(454),
+      ("454.0", "000D0") -> Decimal(454),
+      ("454.00", "999.99") -> Decimal(454),
+      ("454.00", "000.00") -> Decimal(454),
+      ("454.00", "999D99") -> Decimal(454),
+      ("454.00", "000D00") -> Decimal(454),
+      (".4542", ".9999") -> Decimal(0.4542),
+      (".4542", ".0000") -> Decimal(0.4542),
+      (".4542", "D9999") -> Decimal(0.4542),
+      (".4542", "D0000") -> Decimal(0.4542),
+      ("4542.", "9999.") -> Decimal(4542),
+      ("4542.", "0000.") -> Decimal(4542),
+      ("4542.", "9999D") -> Decimal(4542),
+      ("4542.", "0000D") -> Decimal(4542)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    Seq("999.9.9", "999D9D9", "999.9D9", "999D9.9").foreach { str =>
+      ToNumber(Literal("454.3.2"), Literal(str)).checkInputDataTypes() match {
+        case TypeCheckResult.TypeCheckFailure(msg) =>
+          assert(msg.contains(s"Multiple 'D' or '.' in '$str'"))
+      }
+    }
+
+    // Test ',' and 'G'
+    Seq(
+      ("12,454", "99,999") -> Decimal(12454),
+      ("12,454", "00,000") -> Decimal(12454),
+      ("12,454", "99G999") -> Decimal(12454),
+      ("12,454", "00G000") -> Decimal(12454),
+      ("12,454,367", "99,999,999") -> Decimal(12454367),
+      ("12,454,367", "00,000,000") -> Decimal(12454367),
+      ("12,454,367", "99G999G999") -> Decimal(12454367),
+      ("12,454,367", "00G000G000") -> Decimal(12454367),
+      ("12,454,", "99,999,") -> Decimal(12454),
+      ("12,454,", "00,000,") -> Decimal(12454),
+      ("12,454,", "99G999G") -> Decimal(12454),
+      ("12,454,", "00G000G") -> Decimal(12454),
+      (",454,367", ",999,999") -> Decimal(454367),

Review comment:
       can we test them?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781182350



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -77,57 +101,61 @@ object NumberUtils {
       signIndex > 0 && signIndex < format.length - 1
     }
 
-    if (normalizedFormat.count(_ == pointSign) > 1) {
+    if (normalizedFormat.length == 0) {
+      throw QueryCompilationErrors.emptyNumberFormatError()
+    } else if (normalizedFormat.count(_ == POINT_SIGN) > 1) {
       throw QueryCompilationErrors.multipleSignInNumberFormatError(
-        s"'$letterPointSign' or '$pointSign'", numberFormat)
-    } else if (normalizedFormat.count(_ == minusSign) > 1) {
+        s"'$POINT_LETTER' or '$POINT_SIGN'", numberFormat)
+    } else if (normalizedFormat.count(_ == MINUS_SIGN) > 1) {
       throw QueryCompilationErrors.multipleSignInNumberFormatError(
-        s"'$letterMinusSign' or '$minusSign'", numberFormat)
-    } else if (normalizedFormat.count(_ == dollarSign) > 1) {
-      throw QueryCompilationErrors.multipleSignInNumberFormatError(s"'$dollarSign'", numberFormat)
-    } else if (invalidSignPosition(normalizedFormat, minusSign)) {
+        s"'$MINUS_LETTER' or '$MINUS_SIGN'", numberFormat)
+    } else if (normalizedFormat.count(_ == DOLLAR_SIGN) > 1) {
+      throw QueryCompilationErrors.multipleSignInNumberFormatError(s"'$DOLLAR_SIGN'", numberFormat)
+    } else if (invalidSignPosition(normalizedFormat, MINUS_SIGN)) {
       throw QueryCompilationErrors.nonFistOrLastCharInNumberFormatError(
-        s"'$letterMinusSign' or '$minusSign'", numberFormat)
-    } else if (invalidSignPosition(normalizedFormat, dollarSign)) {
+        s"'$MINUS_LETTER' or '$MINUS_SIGN'", numberFormat)
+    } else if (invalidSignPosition(normalizedFormat, DOLLAR_SIGN)) {
       throw QueryCompilationErrors.nonFistOrLastCharInNumberFormatError(
-        s"'$dollarSign'", numberFormat)
+        s"'$DOLLAR_SIGN'", numberFormat)
+    }
+  }
+
+  private def getPrecision(numberFormat: String): Int =
+    numberFormat.filterNot(isSign).length
+
+  private def getScale(numberFormat: String): Int = {
+    val formatSplits = numberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
     }
   }
 
   /**
    * Convert string to numeric based on the given number format.
    * The format can consist of the following characters:
-   * '9':  digit position (can be dropped if insignificant)
-   * '0':  digit position (will not be dropped, even if insignificant)
-   * '.':  decimal point (only allowed once)
-   * ',':  group (thousands) separator
-   * 'S':  sign anchored to number (uses locale)
-   * 'D':  decimal point (uses locale)
-   * 'G':  group separator (uses locale)
-   * '$':  specifies that the input value has a leading $ (Dollar) sign.
+   * '0' or '9': digit position
+   * '.' or 'D': decimal point (only allowed once)
+   * ',' or 'G': group (thousands) separator
+   * '-' or 'S': sign anchored to number
+   * '$': returns value with a leading dollar sign
    *
    * @param input the string need to converted
    * @param numberFormat the given number format
+   * @param normalizedNumberFormat normalized number format
+   * @param precision decimal precision
+   * @param scale decimal scale
    * @return decimal obtained from string parsing
    */
-  def parse(input: UTF8String, numberFormat: String): Decimal = {
-    val normalizedFormat = normalize(numberFormat)
-    check(normalizedFormat, numberFormat)
-
-    val precision = normalizedFormat.filterNot(isSign).length
-    val formatSplits = normalizedFormat.split(pointSign)
-    val scale = if (formatSplits.length == 1) {
-      0
-    } else {
-      formatSplits(1).filterNot(isSign).length
-    }
-    val transformedFormat = transform(normalizedFormat)
-    val numberFormatInstance = NumberFormat.getInstance()
-    val numberDecimalFormat = numberFormatInstance.asInstanceOf[DecimalFormat]
-    numberDecimalFormat.setParseBigDecimal(true)
-    numberDecimalFormat.applyPattern(transformedFormat)
+  private def parse(
+      input: UTF8String,
+      numberFormat: String,
+      normalizedNumberFormat: String,
+      precision: Int,
+      scale: Int): Decimal = {
     val inputStr = input.toString.trim
-    val inputSplits = inputStr.split(pointSign)
+    val inputSplits = inputStr.split(POINT_SIGN)

Review comment:
       what if the input has more than one dot?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781181721



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -77,57 +101,61 @@ object NumberUtils {
       signIndex > 0 && signIndex < format.length - 1
     }
 
-    if (normalizedFormat.count(_ == pointSign) > 1) {
+    if (normalizedFormat.length == 0) {
+      throw QueryCompilationErrors.emptyNumberFormatError()
+    } else if (normalizedFormat.count(_ == POINT_SIGN) > 1) {
       throw QueryCompilationErrors.multipleSignInNumberFormatError(
-        s"'$letterPointSign' or '$pointSign'", numberFormat)
-    } else if (normalizedFormat.count(_ == minusSign) > 1) {
+        s"'$POINT_LETTER' or '$POINT_SIGN'", numberFormat)
+    } else if (normalizedFormat.count(_ == MINUS_SIGN) > 1) {
       throw QueryCompilationErrors.multipleSignInNumberFormatError(
-        s"'$letterMinusSign' or '$minusSign'", numberFormat)
-    } else if (normalizedFormat.count(_ == dollarSign) > 1) {
-      throw QueryCompilationErrors.multipleSignInNumberFormatError(s"'$dollarSign'", numberFormat)
-    } else if (invalidSignPosition(normalizedFormat, minusSign)) {
+        s"'$MINUS_LETTER' or '$MINUS_SIGN'", numberFormat)
+    } else if (normalizedFormat.count(_ == DOLLAR_SIGN) > 1) {
+      throw QueryCompilationErrors.multipleSignInNumberFormatError(s"'$DOLLAR_SIGN'", numberFormat)
+    } else if (invalidSignPosition(normalizedFormat, MINUS_SIGN)) {
       throw QueryCompilationErrors.nonFistOrLastCharInNumberFormatError(
-        s"'$letterMinusSign' or '$minusSign'", numberFormat)
-    } else if (invalidSignPosition(normalizedFormat, dollarSign)) {
+        s"'$MINUS_LETTER' or '$MINUS_SIGN'", numberFormat)
+    } else if (invalidSignPosition(normalizedFormat, DOLLAR_SIGN)) {
       throw QueryCompilationErrors.nonFistOrLastCharInNumberFormatError(
-        s"'$dollarSign'", numberFormat)
+        s"'$DOLLAR_SIGN'", numberFormat)
+    }
+  }
+
+  private def getPrecision(numberFormat: String): Int =
+    numberFormat.filterNot(isSign).length
+
+  private def getScale(numberFormat: String): Int = {
+    val formatSplits = numberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
     }
   }
 
   /**
    * Convert string to numeric based on the given number format.
    * The format can consist of the following characters:
-   * '9':  digit position (can be dropped if insignificant)
-   * '0':  digit position (will not be dropped, even if insignificant)
-   * '.':  decimal point (only allowed once)
-   * ',':  group (thousands) separator
-   * 'S':  sign anchored to number (uses locale)
-   * 'D':  decimal point (uses locale)
-   * 'G':  group separator (uses locale)
-   * '$':  specifies that the input value has a leading $ (Dollar) sign.
+   * '0' or '9': digit position
+   * '.' or 'D': decimal point (only allowed once)
+   * ',' or 'G': group (thousands) separator
+   * '-' or 'S': sign anchored to number
+   * '$': returns value with a leading dollar sign
    *
    * @param input the string need to converted
    * @param numberFormat the given number format
+   * @param normalizedNumberFormat normalized number format
+   * @param precision decimal precision
+   * @param scale decimal scale
    * @return decimal obtained from string parsing
    */
-  def parse(input: UTF8String, numberFormat: String): Decimal = {
-    val normalizedFormat = normalize(numberFormat)
-    check(normalizedFormat, numberFormat)
-
-    val precision = normalizedFormat.filterNot(isSign).length
-    val formatSplits = normalizedFormat.split(pointSign)
-    val scale = if (formatSplits.length == 1) {
-      0
-    } else {
-      formatSplits(1).filterNot(isSign).length
-    }
-    val transformedFormat = transform(normalizedFormat)
-    val numberFormatInstance = NumberFormat.getInstance()
-    val numberDecimalFormat = numberFormatInstance.asInstanceOf[DecimalFormat]
-    numberDecimalFormat.setParseBigDecimal(true)
-    numberDecimalFormat.applyPattern(transformedFormat)
+  private def parse(
+      input: UTF8String,
+      numberFormat: String,

Review comment:
       `originalNumberFormat`?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784467972



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.sql.AnalysisException
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatter(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private lazy val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {
+    val decimalFormat = new DecimalFormat(transformedFormat)
+    decimalFormat.setParseBigDecimal(true)
+    decimalFormat
+  }
+
+  private lazy val precision = normalizedNumberFormat.filterNot(isSign).length
+
+  private lazy val scale = {
+    val formatSplits = normalizedNumberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
+    }
+  }
+
+  def parsedDecimalType: DecimalType = DecimalType(precision, scale)
+
+  /**
+   * DecimalFormat provides '#' and '0' as placeholder of digit, ',' as grouping separator,
+   * '.' as decimal separator, '-' as minus, '$' as dollar, but not '9', 'G', 'D', 'S'. So we need
+   * replace them show below:
+   * 1. '9' -> '#'
+   * 2. 'G' -> ','
+   * 3. 'D' -> '.'
+   * 4. 'S' -> '-'
+   *
+   * Note: When calling format, we must preserve the digits after decimal point, so the digits
+   * after decimal point should be replaced as '0'. For example: '999.9' will be normalized as
+   * '###.0' and '999.99' will be normalized as '###.00', so if the input is 454, the format
+   * output will be 454.0 and 454.00 respectively.
+   *
+   * @param format number format string
+   * @return normalized number format string
+   */
+  private def normalize(format: String): String = {
+    var notFindDecimalPoint = true
+    val normalizedFormat = format.toUpperCase(Locale.ROOT).map {
+      case NINE_DIGIT if notFindDecimalPoint => POUND_SIGN
+      case NINE_DIGIT if !notFindDecimalPoint => ZERO_DIGIT
+      case COMMA_LETTER => COMMA_SIGN
+      case POINT_LETTER | POINT_SIGN =>
+        notFindDecimalPoint = false
+        POINT_SIGN
+      case MINUS_LETTER => MINUS_SIGN
+      case other => other
+    }
+    // If the comma is at the beginning or end of number format, then DecimalFormat will be
+    // invalid. For example, "##,###," or ",###,###" for DecimalFormat is invalid, so we must use
+    // "##,###" or "###,###".
+    normalizedFormat.stripPrefix(COMMA_SIGN_STRING).stripSuffix(COMMA_SIGN_STRING)
+  }
+
+  private def isSign(c: Char): Boolean = {
+    SIGN_SET.contains(c)
+  }
+
+  private def transform(format: String): String = {
+    if (format.contains(MINUS_SIGN)) {
+      // For example: '#.######' represents a positive number,
+      // but '#.######;#.######-' represents a negative number.
+      val positiveFormatString = format.replaceAll("-", "")
+      s"$positiveFormatString;$format"
+    } else {
+      format
+    }
+  }
+
+  def check(): TypeCheckResult = {
+    def invalidSignPosition(c: Char): Boolean = {
+      val signIndex = normalizedNumberFormat.indexOf(c)
+      signIndex > 0 && signIndex < normalizedNumberFormat.length - 1
+    }
+
+    def multipleSignInNumberFormatError(message: String): String = {
+      s"Multiple $message in '$originNumberFormat'"
+    }
+
+    def nonFistOrLastCharInNumberFormatError(message: String): String = {
+      s"$message must be the first or last char in '$originNumberFormat'"

Review comment:
       ```suggestion
         s"$message must be the first or last char in the number format: '$originNumberFormat'"
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776909773



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)
+         '.':  decimal point (only allowed once)
+         ',':  group (thousands) separator
+         'S':  sign anchored to number (uses locale)
+         'D':  decimal point (uses locale)

Review comment:
       This comment references `PostgreSQL` and `Oracle`. But other database does not mention locale.
   I guess `NumberFormat.getNumberInstance(Locale.ROOT)` is enough.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781167049



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '0' or '9': digit position
+         '.' or 'D':  decimal point (only allowed once)
+         ',' or 'G':  group (thousands) separator
+         '-' or 'S':  sign anchored to number

Review comment:
       is it only allowed once?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784507815



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.sql.AnalysisException
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatter(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private lazy val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {

Review comment:
       OK




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786117500



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,179 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+
+    Seq("454", "054", "54", "450").foreach { input =>
+      val invalidFormat1 = 0.until(input.length - 1).map(_ => '0').mkString
+      val invalidFormat2 = 0.until(input.length - 2).map(_ => '0').mkString
+      val invalidFormat3 = 0.until(input.length - 1).map(_ => '9').mkString
+      val invalidFormat4 = 0.until(input.length - 2).map(_ => '9').mkString
+      Seq(invalidFormat1, invalidFormat2, invalidFormat3, invalidFormat4)
+        .filter(_.nonEmpty).foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454"), Literal(format)), s"Format '$format' used for" +
+            " parsing string to number or formatting number to string is invalid")
+      }
+
+      val format1 = 0.until(input.length).map(_ => '0').mkString
+      val format2 = 0.until(input.length).map(_ => '9').mkString
+      val format3 = 0.until(input.length + 1).map(_ => '0').mkString
+      val format4 = 0.until(input.length + 1).map(_ => '9').mkString
+      val format5 = 0.until(input.length + 2).map(_ => '0').mkString
+      val format6 = 0.until(input.length + 2).map(_ => '9').mkString
+      Seq(format1, format2, format3, format4, format5, format6).foreach { format =>
+        checkEvaluation(ToNumber(Literal(input), Literal(format)), Decimal(input))
+      }
+    }
+
+    // Test '.' and 'D'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.2"), Literal("999")),
+      "Format '999' used for parsing string to number or formatting number to string is invalid")

Review comment:
       I think we should refine the error message. `999` is indeed a valid number format string. The problem is the input does not match it.
   
   How about
   ```
   The input string '$input' does not match the given number format: '$format'
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783643127



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,174 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("99")))
+    }
+
+    Seq(
+      ("454", "999") -> Decimal(454),
+      ("054", "999") -> Decimal(54),
+      ("54", "999") -> Decimal(54),
+      ("404", "999") -> Decimal(404),
+      ("450", "999") -> Decimal(450),
+      ("454", "9999") -> Decimal(454),
+      ("054", "9999") -> Decimal(54),
+      ("404", "9999") -> Decimal(404),
+      ("450", "9999") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("0")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("00")))
+    }
+
+    Seq(
+      ("454", "000") -> Decimal(454),
+      ("054", "000") -> Decimal(54),
+      ("54", "000") -> Decimal(54),
+      ("404", "000") -> Decimal(404),
+      ("450", "000") -> Decimal(450),
+      ("454", "0000") -> Decimal(454),
+      ("054", "0000") -> Decimal(54),
+      ("404", "0000") -> Decimal(404),
+      ("450", "0000") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    // Test '.' and 'D'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.2"), Literal("999")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.23"), Literal("999.9")))
+    }
+
+    Seq(
+      ("454.2", "999.9") -> Decimal(454.2),

Review comment:
       some suggestions for the tests:
   1. We know that some pattern chars are identical after normalization, and we don't need to test it everywhere. Just put a few tests to verify it.
   2. We should test more factors, e.g. the number of `0` or `9` is equal to, smaller than, or larger than the actual number of digits in the value.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783920174



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -164,26 +192,74 @@ object NumberUtils {
     val bigDecimal = input.toJavaBigDecimal
     val decimalPlainStr = bigDecimal.toPlainString
     if (decimalPlainStr.length > transformedFormat.length) {
-      transformedFormat.replaceAll("0", "#")
+      transformedFormat.replaceAll("0", POUND_SIGN_STRING)
     } else {
-      val decimalFormat = new DecimalFormat(transformedFormat)
-      var resultStr = decimalFormat.format(bigDecimal)
+      val numberDecimalFormat = {
+        val decimalFormat = new DecimalFormat()
+        decimalFormat.setParseBigDecimal(true)
+        try {
+          decimalFormat.applyLocalizedPattern(transformedFormat)
+        } catch {
+          case _: IllegalArgumentException =>
+            throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
+        }
+        decimalFormat
+      }
+      var resultStr = numberDecimalFormat.format(bigDecimal)
       // Since we trimmed the comma at the beginning or end of number format in function
       // `normalize`, we restore the comma to the result here.
       // For example, if the specified number format is "99,999," or ",999,999", function
       // `normalize` normalize them to "##,###" or "###,###".
       // new DecimalFormat("##,###").parse(12454) and new DecimalFormat("###,###").parse(124546)
       // will return "12,454" and "124,546" respectively. So we add ',' at the end and head of
       // the result, then the final output are "12,454," or ",124,546".
-      if (numberFormat.last == commaSign || numberFormat.last == letterCommaSign) {
-        resultStr = resultStr + commaSign
+      if (numberFormat.last == COMMA_SIGN || numberFormat.last == COMMA_LETTER) {
+        resultStr = resultStr + COMMA_SIGN
       }
-      if (numberFormat.charAt(0) == commaSign || numberFormat.charAt(0) == letterCommaSign) {
-        resultStr = commaSign + resultStr
+      if (numberFormat.charAt(0) == COMMA_SIGN || numberFormat.charAt(0) == COMMA_LETTER) {
+        resultStr = COMMA_SIGN + resultStr
       }
 
       resultStr
     }
   }
 
+  class NumberFormatBuilder(originNumberFormat: String) extends Serializable {

Review comment:
       Updated.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan closed pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #35060:
URL: https://github.com/apache/spark/pull/35060


   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776639337



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)
+         '.':  decimal point (only allowed once)
+         ',':  group (thousands) separator
+         'S':  sign anchored to number (uses locale)
+         'D':  decimal point (uses locale)
+         'G':  group separator (uses locale)
+         '$':  specifies that the input value has a leading $ (Dollar) sign.
+  """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_('454', '999');
+       454
+      > SELECT _FUNC_('454.00', '000D00');
+       454.00
+      > SELECT _FUNC_('12,454', '99G999');
+       12454
+      > SELECT _FUNC_('$78.12', '$99.99');
+       78.12
+      > SELECT _FUNC_('12,454.8-', '99G999D9S');
+       -12454.8
+  """,
+  since = "3.3.0",
+  group = "string_funcs")
+case class ToNumber(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant {
+
+  private lazy val numberFormat = right.eval().toString.toUpperCase(Locale.ROOT)
+  private lazy val builder = new NumberFormatBuilder(numberFormat)
+  private lazy val (precision, scale) = builder.parsePrecisionAndScale()
+
+  override def dataType: DataType = DecimalType(precision, scale)
+
+  override def inputTypes: Seq[DataType] = Seq(StringType, StringType)
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val inputTypeCheck = super.checkInputDataTypes()
+    if (inputTypeCheck.isSuccess) {
+      if (right.foldable) {
+        if (builder.normalizedNumberFormat.length == 0) {
+          TypeCheckResult.TypeCheckFailure(s"Format expression cannot be empty")

Review comment:
       can we do this check in `builder.check()`?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776638876



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)

Review comment:
       I think we need more documents. I still don't understand the difference between 0 and 9 after reading this.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779595366



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0999|
+|**.**|decimal point (only allowed once)|99.99|
+|**D**|decimal point (only allowed once)|99D99|
+|**,**|group (thousands) separator|9,999|
+|**G**|group (thousands) separator|9G999|
+|**-**|sign anchored to number|-9999|

Review comment:
       this is also only allowed once?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784468389



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.sql.AnalysisException
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatter(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private lazy val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {
+    val decimalFormat = new DecimalFormat(transformedFormat)
+    decimalFormat.setParseBigDecimal(true)
+    decimalFormat
+  }
+
+  private lazy val precision = normalizedNumberFormat.filterNot(isSign).length
+
+  private lazy val scale = {
+    val formatSplits = normalizedNumberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
+    }
+  }
+
+  def parsedDecimalType: DecimalType = DecimalType(precision, scale)
+
+  /**
+   * DecimalFormat provides '#' and '0' as placeholder of digit, ',' as grouping separator,
+   * '.' as decimal separator, '-' as minus, '$' as dollar, but not '9', 'G', 'D', 'S'. So we need
+   * replace them show below:
+   * 1. '9' -> '#'
+   * 2. 'G' -> ','
+   * 3. 'D' -> '.'
+   * 4. 'S' -> '-'
+   *
+   * Note: When calling format, we must preserve the digits after decimal point, so the digits
+   * after decimal point should be replaced as '0'. For example: '999.9' will be normalized as
+   * '###.0' and '999.99' will be normalized as '###.00', so if the input is 454, the format
+   * output will be 454.0 and 454.00 respectively.
+   *
+   * @param format number format string
+   * @return normalized number format string
+   */
+  private def normalize(format: String): String = {
+    var notFindDecimalPoint = true
+    val normalizedFormat = format.toUpperCase(Locale.ROOT).map {
+      case NINE_DIGIT if notFindDecimalPoint => POUND_SIGN
+      case NINE_DIGIT if !notFindDecimalPoint => ZERO_DIGIT
+      case COMMA_LETTER => COMMA_SIGN
+      case POINT_LETTER | POINT_SIGN =>
+        notFindDecimalPoint = false
+        POINT_SIGN
+      case MINUS_LETTER => MINUS_SIGN
+      case other => other
+    }
+    // If the comma is at the beginning or end of number format, then DecimalFormat will be
+    // invalid. For example, "##,###," or ",###,###" for DecimalFormat is invalid, so we must use
+    // "##,###" or "###,###".
+    normalizedFormat.stripPrefix(COMMA_SIGN_STRING).stripSuffix(COMMA_SIGN_STRING)
+  }
+
+  private def isSign(c: Char): Boolean = {
+    SIGN_SET.contains(c)
+  }
+
+  private def transform(format: String): String = {
+    if (format.contains(MINUS_SIGN)) {
+      // For example: '#.######' represents a positive number,
+      // but '#.######;#.######-' represents a negative number.
+      val positiveFormatString = format.replaceAll("-", "")
+      s"$positiveFormatString;$format"
+    } else {
+      format
+    }
+  }
+
+  def check(): TypeCheckResult = {
+    def invalidSignPosition(c: Char): Boolean = {
+      val signIndex = normalizedNumberFormat.indexOf(c)
+      signIndex > 0 && signIndex < normalizedNumberFormat.length - 1
+    }
+
+    def multipleSignInNumberFormatError(message: String): String = {
+      s"Multiple $message in '$originNumberFormat'"
+    }
+
+    def nonFistOrLastCharInNumberFormatError(message: String): String = {
+      s"$message must be the first or last char in '$originNumberFormat'"
+    }
+
+    if (normalizedNumberFormat.length == 0) {
+      TypeCheckResult.TypeCheckFailure("Number format cannot be empty")
+    } else if (normalizedNumberFormat.count(_ == POINT_SIGN) > 1) {
+      TypeCheckResult.TypeCheckFailure(
+        multipleSignInNumberFormatError(s"'$POINT_LETTER' or '$POINT_SIGN'"))
+    } else if (normalizedNumberFormat.count(_ == MINUS_SIGN) > 1) {
+      TypeCheckResult.TypeCheckFailure(
+        multipleSignInNumberFormatError(s"'$MINUS_LETTER' or '$MINUS_SIGN'"))
+    } else if (normalizedNumberFormat.count(_ == DOLLAR_SIGN) > 1) {
+      TypeCheckResult.TypeCheckFailure(multipleSignInNumberFormatError(s"'$DOLLAR_SIGN'"))
+    } else if (invalidSignPosition(MINUS_SIGN)) {
+      TypeCheckResult.TypeCheckFailure(
+        nonFistOrLastCharInNumberFormatError(s"'$MINUS_LETTER' or '$MINUS_SIGN'"))
+    } else if (invalidSignPosition(DOLLAR_SIGN)) {

Review comment:
       why do we allow dollar sign at the end? is it true in other databases?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786118941



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,179 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+
+    Seq("454", "054", "54", "450").foreach { input =>
+      val invalidFormat1 = 0.until(input.length - 1).map(_ => '0').mkString
+      val invalidFormat2 = 0.until(input.length - 2).map(_ => '0').mkString
+      val invalidFormat3 = 0.until(input.length - 1).map(_ => '9').mkString
+      val invalidFormat4 = 0.until(input.length - 2).map(_ => '9').mkString
+      Seq(invalidFormat1, invalidFormat2, invalidFormat3, invalidFormat4)
+        .filter(_.nonEmpty).foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454"), Literal(format)), s"Format '$format' used for" +
+            " parsing string to number or formatting number to string is invalid")
+      }
+
+      val format1 = 0.until(input.length).map(_ => '0').mkString
+      val format2 = 0.until(input.length).map(_ => '9').mkString
+      val format3 = 0.until(input.length + 1).map(_ => '0').mkString
+      val format4 = 0.until(input.length + 1).map(_ => '9').mkString
+      val format5 = 0.until(input.length + 2).map(_ => '0').mkString
+      val format6 = 0.until(input.length + 2).map(_ => '9').mkString
+      Seq(format1, format2, format3, format4, format5, format6).foreach { format =>
+        checkEvaluation(ToNumber(Literal(input), Literal(format)), Decimal(input))
+      }
+    }
+
+    // Test '.' and 'D'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.2"), Literal("999")),
+      "Format '999' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("999.9")),
+      "Format '999.9' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("000.0")),
+      "Format '000.0' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("000D0")),
+      "Format '000D0' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("00.00")),
+      "Format '00.00' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("00D00")),
+      "Format '00D00' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("0000.0")),
+      "Format '0000.0' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("0000D0")),
+      "Format '0000D0' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("00.000")),
+      "Format '00.000' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("00D000")),

Review comment:
       if you really want to test all the cases, please write the test cases only with dot, and run it again after replacing dot with `D`.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784469313



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.sql.AnalysisException
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatter(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private lazy val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {
+    val decimalFormat = new DecimalFormat(transformedFormat)
+    decimalFormat.setParseBigDecimal(true)
+    decimalFormat
+  }
+
+  private lazy val precision = normalizedNumberFormat.filterNot(isSign).length
+
+  private lazy val scale = {
+    val formatSplits = normalizedNumberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
+    }
+  }
+
+  def parsedDecimalType: DecimalType = DecimalType(precision, scale)
+
+  /**
+   * DecimalFormat provides '#' and '0' as placeholder of digit, ',' as grouping separator,
+   * '.' as decimal separator, '-' as minus, '$' as dollar, but not '9', 'G', 'D', 'S'. So we need
+   * replace them show below:
+   * 1. '9' -> '#'
+   * 2. 'G' -> ','
+   * 3. 'D' -> '.'
+   * 4. 'S' -> '-'
+   *
+   * Note: When calling format, we must preserve the digits after decimal point, so the digits
+   * after decimal point should be replaced as '0'. For example: '999.9' will be normalized as
+   * '###.0' and '999.99' will be normalized as '###.00', so if the input is 454, the format
+   * output will be 454.0 and 454.00 respectively.
+   *
+   * @param format number format string
+   * @return normalized number format string
+   */
+  private def normalize(format: String): String = {
+    var notFindDecimalPoint = true
+    val normalizedFormat = format.toUpperCase(Locale.ROOT).map {
+      case NINE_DIGIT if notFindDecimalPoint => POUND_SIGN
+      case NINE_DIGIT if !notFindDecimalPoint => ZERO_DIGIT
+      case COMMA_LETTER => COMMA_SIGN
+      case POINT_LETTER | POINT_SIGN =>
+        notFindDecimalPoint = false
+        POINT_SIGN
+      case MINUS_LETTER => MINUS_SIGN
+      case other => other
+    }
+    // If the comma is at the beginning or end of number format, then DecimalFormat will be
+    // invalid. For example, "##,###," or ",###,###" for DecimalFormat is invalid, so we must use
+    // "##,###" or "###,###".
+    normalizedFormat.stripPrefix(COMMA_SIGN_STRING).stripSuffix(COMMA_SIGN_STRING)
+  }
+
+  private def isSign(c: Char): Boolean = {
+    SIGN_SET.contains(c)
+  }
+
+  private def transform(format: String): String = {
+    if (format.contains(MINUS_SIGN)) {
+      // For example: '#.######' represents a positive number,
+      // but '#.######;#.######-' represents a negative number.
+      val positiveFormatString = format.replaceAll("-", "")
+      s"$positiveFormatString;$format"
+    } else {
+      format
+    }
+  }
+
+  def check(): TypeCheckResult = {
+    def invalidSignPosition(c: Char): Boolean = {
+      val signIndex = normalizedNumberFormat.indexOf(c)
+      signIndex > 0 && signIndex < normalizedNumberFormat.length - 1
+    }
+
+    def multipleSignInNumberFormatError(message: String): String = {
+      s"Multiple $message in '$originNumberFormat'"
+    }
+
+    def nonFistOrLastCharInNumberFormatError(message: String): String = {
+      s"$message must be the first or last char in '$originNumberFormat'"
+    }
+
+    if (normalizedNumberFormat.length == 0) {
+      TypeCheckResult.TypeCheckFailure("Number format cannot be empty")
+    } else if (normalizedNumberFormat.count(_ == POINT_SIGN) > 1) {
+      TypeCheckResult.TypeCheckFailure(
+        multipleSignInNumberFormatError(s"'$POINT_LETTER' or '$POINT_SIGN'"))
+    } else if (normalizedNumberFormat.count(_ == MINUS_SIGN) > 1) {
+      TypeCheckResult.TypeCheckFailure(
+        multipleSignInNumberFormatError(s"'$MINUS_LETTER' or '$MINUS_SIGN'"))
+    } else if (normalizedNumberFormat.count(_ == DOLLAR_SIGN) > 1) {
+      TypeCheckResult.TypeCheckFailure(multipleSignInNumberFormatError(s"'$DOLLAR_SIGN'"))
+    } else if (invalidSignPosition(MINUS_SIGN)) {
+      TypeCheckResult.TypeCheckFailure(
+        nonFistOrLastCharInNumberFormatError(s"'$MINUS_LETTER' or '$MINUS_SIGN'"))
+    } else if (invalidSignPosition(DOLLAR_SIGN)) {
+      TypeCheckResult.TypeCheckFailure(
+        nonFistOrLastCharInNumberFormatError(s"'$DOLLAR_SIGN'"))
+    } else {
+      TypeCheckResult.TypeCheckSuccess
+    }
+  }
+
+  /**
+   * Convert string to numeric based on the given number format.
+   * The format can consist of the following characters:
+   * '0' or '9': digit position
+   * '.' or 'D': decimal point (only allowed once)
+   * ',' or 'G': group (thousands) separator
+   * '-' or 'S': sign anchored to number (only allowed once)
+   * '$': value with a leading dollar sign (only allowed once)
+   *
+   * @param input the string need to converted
+   * @return decimal obtained from string parsing
+   */
+  def parse(input: UTF8String): Decimal = {
+    val inputStr = input.toString.trim
+    val inputSplits = inputStr.split(POINT_SIGN)
+    if (inputSplits.length == 1) {
+      if (inputStr.filterNot(isSign).length > precision - scale) {
+        throw QueryExecutionErrors.invalidNumberFormatError(originNumberFormat)
+      }
+    } else if (inputSplits(0).filterNot(isSign).length > precision - scale ||
+      inputSplits(1).filterNot(isSign).length > scale) {
+      throw QueryExecutionErrors.invalidNumberFormatError(originNumberFormat)
+    }
+
+    val number = numberDecimalFormat.parse(inputStr, new ParsePosition(0))
+    assert(number.isInstanceOf[BigDecimal])
+    Decimal(number.asInstanceOf[BigDecimal])
+  }
+
+  /**
+   * Convert numeric to string based on the given number format.
+   * The format can consist of the following characters:
+   * '9': digit position (can be dropped if insignificant)
+   * '0': digit position (will not be dropped, even if insignificant)
+   * '.' or 'D': decimal point (only allowed once)
+   * ',' or 'G': group (thousands) separator
+   * '-' or 'S': sign anchored to number (only allowed once)
+   * '$': value with a leading dollar sign (only allowed once)
+   *
+   * @param input the decimal to format
+   * @param numberFormat the format string
+   * @return The string after formatting input decimal
+   */
+  def format(input: Decimal): String = {
+    val bigDecimal = input.toJavaBigDecimal
+    val decimalPlainStr = bigDecimal.toPlainString
+    if (decimalPlainStr.length > transformedFormat.length) {
+      transformedFormat.replaceAll("0", POUND_SIGN_STRING)
+    } else {
+      val numberDecimalFormat = {
+        val decimalFormat = new DecimalFormat()
+        decimalFormat.setParseBigDecimal(true)
+        try {
+          decimalFormat.applyLocalizedPattern(transformedFormat)

Review comment:
       do we still need this?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r787795300



##########
File path: sql/core/src/test/resources/sql-tests/inputs/string-functions.sql
##########
@@ -124,4 +124,22 @@ SELECT endswith('Spark SQL', 'QL');
 SELECT endswith('Spark SQL', 'Spa');
 SELECT endswith(null, 'Spark');
 SELECT endswith('Spark', null);
-SELECT endswith(null, null);
\ No newline at end of file
+SELECT endswith(null, null);
+
+-- to_number
+select to_number('454', '999');
+select to_number('454', '000');
+select to_number('454.2', '999.9');
+select to_number('454.2', '000.0');
+select to_number('454.2', '999D9');

Review comment:
       ditto for `.` and `D`

##########
File path: sql/core/src/test/resources/sql-tests/inputs/string-functions.sql
##########
@@ -124,4 +124,22 @@ SELECT endswith('Spark SQL', 'QL');
 SELECT endswith('Spark SQL', 'Spa');
 SELECT endswith(null, 'Spark');
 SELECT endswith('Spark', null);
-SELECT endswith(null, null);
\ No newline at end of file
+SELECT endswith(null, null);
+
+-- to_number
+select to_number('454', '999');
+select to_number('454', '000');
+select to_number('454.2', '999.9');
+select to_number('454.2', '000.0');
+select to_number('454.2', '999D9');
+select to_number('454.2', '000D0');
+select to_number('12,454', '99,999');
+select to_number('12,454', '00,000');
+select to_number('12,454', '99G999');

Review comment:
       ditto

##########
File path: sql/core/src/test/resources/sql-tests/inputs/string-functions.sql
##########
@@ -124,4 +124,22 @@ SELECT endswith('Spark SQL', 'QL');
 SELECT endswith('Spark SQL', 'Spa');
 SELECT endswith(null, 'Spark');
 SELECT endswith('Spark', null);
-SELECT endswith(null, null);
\ No newline at end of file
+SELECT endswith(null, null);
+
+-- to_number
+select to_number('454', '999');
+select to_number('454', '000');
+select to_number('454.2', '999.9');
+select to_number('454.2', '000.0');
+select to_number('454.2', '999D9');
+select to_number('454.2', '000D0');
+select to_number('12,454', '99,999');
+select to_number('12,454', '00,000');
+select to_number('12,454', '99G999');
+select to_number('12,454', '00G000');
+select to_number('$78.12', '$99.99');
+select to_number('$78.12', '$00.00');
+select to_number('-454', '-999');
+select to_number('-454', 'S999');

Review comment:
       ditto




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r787793732



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberFormatterSuite.scala
##########
@@ -19,76 +19,75 @@ package org.apache.spark.sql.catalyst.util
 
 import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.util.NumberUtils.{format, parse}
 import org.apache.spark.sql.types.Decimal
 import org.apache.spark.unsafe.types.UTF8String
 
-class NumberUtilsSuite extends SparkFunSuite {
+class NumberFormatterSuite extends SparkFunSuite {

Review comment:
       not sure if we still need this test suite since we test almost everything in the new test in `StringExpressionsSuite`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781166844



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '0' or '9': digit position

Review comment:
       ```suggestion
            '0' or '9':  digit position
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781187066



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,172 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("99")))
+    }
+
+    Seq(
+      ("454", "999") -> Decimal(454),
+      ("054", "999") -> Decimal(54),

Review comment:
       how about `"54"` and `"999"`?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781681382



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '0' or '9': digit position
+         '.' or 'D':  decimal point (only allowed once)
+         ',' or 'G':  group (thousands) separator
+         '-' or 'S':  sign anchored to number

Review comment:
       PostgreSQL throws `ERROR: cannot use "S" twice`
   Oracle throws `ORA-01481: 无效的数字格式模型.`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783642153



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -164,26 +192,74 @@ object NumberUtils {
     val bigDecimal = input.toJavaBigDecimal
     val decimalPlainStr = bigDecimal.toPlainString
     if (decimalPlainStr.length > transformedFormat.length) {
-      transformedFormat.replaceAll("0", "#")
+      transformedFormat.replaceAll("0", POUND_SIGN_STRING)
     } else {
-      val decimalFormat = new DecimalFormat(transformedFormat)
-      var resultStr = decimalFormat.format(bigDecimal)
+      val numberDecimalFormat = {
+        val decimalFormat = new DecimalFormat()
+        decimalFormat.setParseBigDecimal(true)
+        try {
+          decimalFormat.applyLocalizedPattern(transformedFormat)
+        } catch {
+          case _: IllegalArgumentException =>
+            throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
+        }
+        decimalFormat
+      }
+      var resultStr = numberDecimalFormat.format(bigDecimal)
       // Since we trimmed the comma at the beginning or end of number format in function
       // `normalize`, we restore the comma to the result here.
       // For example, if the specified number format is "99,999," or ",999,999", function
       // `normalize` normalize them to "##,###" or "###,###".
       // new DecimalFormat("##,###").parse(12454) and new DecimalFormat("###,###").parse(124546)
       // will return "12,454" and "124,546" respectively. So we add ',' at the end and head of
       // the result, then the final output are "12,454," or ",124,546".
-      if (numberFormat.last == commaSign || numberFormat.last == letterCommaSign) {
-        resultStr = resultStr + commaSign
+      if (numberFormat.last == COMMA_SIGN || numberFormat.last == COMMA_LETTER) {
+        resultStr = resultStr + COMMA_SIGN
       }
-      if (numberFormat.charAt(0) == commaSign || numberFormat.charAt(0) == letterCommaSign) {
-        resultStr = commaSign + resultStr
+      if (numberFormat.charAt(0) == COMMA_SIGN || numberFormat.charAt(0) == COMMA_LETTER) {
+        resultStr = COMMA_SIGN + resultStr
       }
 
       resultStr
     }
   }
 
+  class NumberFormatBuilder(originNumberFormat: String) extends Serializable {

Review comment:
       What's your final code structure will be? It's unclear to me when we need to put code in `NumberUtils`, and when in this class.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779983528



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0999|
+|**.**|decimal point (only allowed once)|99.99|
+|**D**|decimal point (only allowed once)|99D99|
+|**,**|group (thousands) separator|9,999|
+|**G**|group (thousands) separator|9G999|
+|**-**|sign anchored to number|-9999|

Review comment:
       Yes




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783643597



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,174 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("99")))
+    }
+
+    Seq(
+      ("454", "999") -> Decimal(454),
+      ("054", "999") -> Decimal(54),
+      ("54", "999") -> Decimal(54),
+      ("404", "999") -> Decimal(404),
+      ("450", "999") -> Decimal(450),
+      ("454", "9999") -> Decimal(454),
+      ("054", "9999") -> Decimal(54),
+      ("404", "9999") -> Decimal(404),
+      ("450", "9999") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("0")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("00")))
+    }
+
+    Seq(
+      ("454", "000") -> Decimal(454),
+      ("054", "000") -> Decimal(54),
+      ("54", "000") -> Decimal(54),
+      ("404", "000") -> Decimal(404),
+      ("450", "000") -> Decimal(450),
+      ("454", "0000") -> Decimal(454),
+      ("054", "0000") -> Decimal(54),
+      ("404", "0000") -> Decimal(404),
+      ("450", "0000") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    // Test '.' and 'D'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.2"), Literal("999")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.23"), Literal("999.9")))
+    }
+
+    Seq(
+      ("454.2", "999.9") -> Decimal(454.2),
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "999D9") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),
+      ("454.23", "999.99") -> Decimal(454.23),
+      ("454.23", "000.00") -> Decimal(454.23),
+      ("454.23", "999D99") -> Decimal(454.23),
+      ("454.23", "000D00") -> Decimal(454.23),
+      ("454.0", "999.9") -> Decimal(454),
+      ("454.0", "000.0") -> Decimal(454),
+      ("454.0", "999D9") -> Decimal(454),
+      ("454.0", "000D0") -> Decimal(454),
+      ("454.00", "999.99") -> Decimal(454),
+      ("454.00", "000.00") -> Decimal(454),
+      ("454.00", "999D99") -> Decimal(454),
+      ("454.00", "000D00") -> Decimal(454),
+      (".4542", ".9999") -> Decimal(0.4542),
+      (".4542", ".0000") -> Decimal(0.4542),
+      (".4542", "D9999") -> Decimal(0.4542),
+      (".4542", "D0000") -> Decimal(0.4542),
+      ("4542.", "9999.") -> Decimal(4542),
+      ("4542.", "0000.") -> Decimal(4542),
+      ("4542.", "9999D") -> Decimal(4542),
+      ("4542.", "0000D") -> Decimal(4542)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    Seq("999.9.9", "999D9D9", "999.9D9", "999D9.9").foreach { str =>
+      ToNumber(Literal("454.3.2"), Literal(str)).checkInputDataTypes() match {
+        case TypeCheckResult.TypeCheckFailure(msg) =>
+          assert(msg.contains(s"Multiple 'D' or '.' in '$str'"))
+      }
+    }
+
+    // Test ',' and 'G'
+    Seq(
+      ("12,454", "99,999") -> Decimal(12454),
+      ("12,454", "00,000") -> Decimal(12454),
+      ("12,454", "99G999") -> Decimal(12454),
+      ("12,454", "00G000") -> Decimal(12454),
+      ("12,454,367", "99,999,999") -> Decimal(12454367),
+      ("12,454,367", "00,000,000") -> Decimal(12454367),
+      ("12,454,367", "99G999G999") -> Decimal(12454367),
+      ("12,454,367", "00G000G000") -> Decimal(12454367),
+      ("12,454,", "99,999,") -> Decimal(12454),
+      ("12,454,", "00,000,") -> Decimal(12454),
+      ("12,454,", "99G999G") -> Decimal(12454),
+      ("12,454,", "00G000G") -> Decimal(12454),
+      (",454,367", ",999,999") -> Decimal(454367),

Review comment:
       is `,` or `G` optional? e.g. does `(",454,367", "999,999")` work? how about `(",454,367", "999,999")`? we can further test number of digits `("123,456", "9G9")`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779990078



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0999|
+|**.**|decimal point (only allowed once)|99.99|
+|**D**|decimal point (only allowed once)|99D99|
+|**,**|group (thousands) separator|9,999|
+|**G**|group (thousands) separator|9G999|
+|**-**|sign anchored to number|-9999|
+|**S**|sign anchored to number|S9999|
+|**$**|returns value with a leading dollar sign|$9999|
+
+Usage notes for numeric formatting:
+
+- 0 specifies a digit position that will always be printed, even if it contains a leading/trailing zero. 9 also specifies a digit position, but if it is a leading zero then it will be replaced by a space, while if it is a trailing zero and fill mode is specified then it will be deleted. (For to_number(), these two pattern characters are equivalent.)
+
+- The pattern characters S, D, and G represent the sign, decimal point, and thousands separator characters. The pattern characters period and comma represent those exact characters, with the meanings of decimal point and thousands separator.

Review comment:
       OK




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r780091912



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,44 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0000|

Review comment:
       similar for `9`
   ```
   Position for a digit; When formatting, it adds a leading/trailing blank space.
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779597666



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0999|
+|**.**|decimal point (only allowed once)|99.99|
+|**D**|decimal point (only allowed once)|99D99|
+|**,**|group (thousands) separator|9,999|
+|**G**|group (thousands) separator|9G999|
+|**-**|sign anchored to number|-9999|
+|**S**|sign anchored to number|S9999|
+|**$**|returns value with a leading dollar sign|$9999|
+
+Usage notes for numeric formatting:
+
+- 0 specifies a digit position that will always be printed, even if it contains a leading/trailing zero. 9 also specifies a digit position, but if it is a leading zero then it will be replaced by a space, while if it is a trailing zero and fill mode is specified then it will be deleted. (For to_number(), these two pattern characters are equivalent.)
+
+- The pattern characters S, D, and G represent the sign, decimal point, and thousands separator characters. The pattern characters period and comma represent those exact characters, with the meanings of decimal point and thousands separator.

Review comment:
       This doesn't say anything. Can we remove it?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779598324



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0999|
+|**.**|decimal point (only allowed once)|99.99|
+|**D**|decimal point (only allowed once)|99D99|
+|**,**|group (thousands) separator|9,999|
+|**G**|group (thousands) separator|9G999|

Review comment:
       ```suggestion
   |**G**|group (thousands) separator, same as **,**|9G999|
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784473607



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,182 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("99")))
+    }
+
+    Seq(
+      ("454", "999") -> Decimal(454),
+      ("054", "999") -> Decimal(54),
+      ("54", "999") -> Decimal(54),
+      ("404", "999") -> Decimal(404),
+      ("450", "999") -> Decimal(450),
+      ("454", "9999") -> Decimal(454),
+      ("054", "9999") -> Decimal(54),
+      ("404", "9999") -> Decimal(404),
+      ("450", "9999") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("0")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("00")))
+    }
+
+    Seq(
+      ("454", "000") -> Decimal(454),
+      ("054", "000") -> Decimal(54),
+      ("54", "000") -> Decimal(54),
+      ("404", "000") -> Decimal(404),
+      ("450", "000") -> Decimal(450),
+      ("454", "0000") -> Decimal(454),
+      ("054", "0000") -> Decimal(54),
+      ("404", "0000") -> Decimal(404),
+      ("450", "0000") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    // Test '.' and 'D'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.2"), Literal("999")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.23"), Literal("999.9")))
+    }
+
+    Seq(
+      ("454.2", "999.9") -> Decimal(454.2),
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "999D9") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),

Review comment:
       please DO NOT repeatedly test the difference between 0 and 9 in all the places. This hurts test readability.
   
   Please only test 0 here and focus on dot in different cases. `.45`, `45.` are good corner cases. We can also test what happens if the number of digits mismatches before and after the dot.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786120642



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,179 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+
+    Seq("454", "054", "54", "450").foreach { input =>
+      val invalidFormat1 = 0.until(input.length - 1).map(_ => '0').mkString
+      val invalidFormat2 = 0.until(input.length - 2).map(_ => '0').mkString
+      val invalidFormat3 = 0.until(input.length - 1).map(_ => '9').mkString
+      val invalidFormat4 = 0.until(input.length - 2).map(_ => '9').mkString
+      Seq(invalidFormat1, invalidFormat2, invalidFormat3, invalidFormat4)
+        .filter(_.nonEmpty).foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454"), Literal(format)), s"Format '$format' used for" +
+            " parsing string to number or formatting number to string is invalid")
+      }
+
+      val format1 = 0.until(input.length).map(_ => '0').mkString
+      val format2 = 0.until(input.length).map(_ => '9').mkString
+      val format3 = 0.until(input.length + 1).map(_ => '0').mkString
+      val format4 = 0.until(input.length + 1).map(_ => '9').mkString
+      val format5 = 0.until(input.length + 2).map(_ => '0').mkString
+      val format6 = 0.until(input.length + 2).map(_ => '9').mkString
+      Seq(format1, format2, format3, format4, format5, format6).foreach { format =>
+        checkEvaluation(ToNumber(Literal(input), Literal(format)), Decimal(input))
+      }
+    }
+
+    // Test '.' and 'D'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.2"), Literal("999")),
+      "Format '999' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("999.9")),
+      "Format '999.9' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("000.0")),
+      "Format '000.0' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("000D0")),
+      "Format '000D0' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("00.00")),
+      "Format '00.00' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("00D00")),
+      "Format '00D00' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("0000.0")),
+      "Format '0000.0' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("0000D0")),
+      "Format '0000D0' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("00.000")),
+      "Format '00.000' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("00D000")),
+      "Format '00D000' used for parsing string to number or formatting number to string is invalid")
+
+    Seq(
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),
+      ("454.23", "000.00") -> Decimal(454.23),
+      ("454.23", "000D00") -> Decimal(454.23),
+      ("454.2", "000.00") -> Decimal(454.2),
+      ("454.2", "000D00") -> Decimal(454.2),
+      ("454.0", "000.0") -> Decimal(454),
+      ("454.0", "000D0") -> Decimal(454),
+      ("454.00", "000.00") -> Decimal(454),
+      ("454.00", "000D00") -> Decimal(454),
+      (".4542", ".0000") -> Decimal(0.4542),
+      (".4542", "D0000") -> Decimal(0.4542),
+      ("4542.", "0000.") -> Decimal(4542),
+      ("4542.", "0000D") -> Decimal(4542)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    Seq("999.9.9", "999D9D9", "999.9D9", "999D9.9").foreach { str =>
+      ToNumber(Literal("454.3.2"), Literal(str)).checkInputDataTypes() match {
+        case TypeCheckResult.TypeCheckFailure(msg) =>
+          assert(msg.contains(s"At most one 'D' or '.' is allowed in the number format: '$str'"))
+      }
+    }
+
+    // Test ',' and 'G'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("123,456"), Literal("9G9")),
+      "Format '9G9' used for parsing string to number or formatting number to string is invalid")
+
+    Seq(
+      ("12,454", "99,999") -> Decimal(12454),
+      ("12,454", "00,000") -> Decimal(12454),
+      ("12,454", "99G999") -> Decimal(12454),
+      ("12,454", "00G000") -> Decimal(12454),

Review comment:
       and tests more cases, e.g. different number of digits.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784005707



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatBuilder(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {
+    val decimalFormat = new DecimalFormat(transformedFormat)
+    decimalFormat.setParseBigDecimal(true)
+    decimalFormat
+  }
+
+  private val precision = getPrecision(normalizedNumberFormat)
+
+  private val scale = getScale(normalizedNumberFormat)
+
+  def parsedDecimalType: DecimalType = DecimalType(precision, scale)

Review comment:
       can we go through `normalizedNumberFormat` once and calculate the decimal type? now we go through it twice and calculate precision and scale separatedly.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r787789178



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,158 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+
+    Seq("454", "054", "54", "450").foreach { input =>
+      val invalidFormat1 = 0.until(input.length - 1).map(_ => '0').mkString
+      val invalidFormat2 = 0.until(input.length - 2).map(_ => '0').mkString
+      val invalidFormat3 = 0.until(input.length - 1).map(_ => '9').mkString
+      val invalidFormat4 = 0.until(input.length - 2).map(_ => '9').mkString
+      Seq(invalidFormat1, invalidFormat2, invalidFormat3, invalidFormat4)
+        .filter(_.nonEmpty).foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal(input), Literal(format)),
+          s"The input string '$input' does not match the given number format: '$format'")
+      }
+
+      val format1 = 0.until(input.length).map(_ => '0').mkString
+      val format2 = 0.until(input.length).map(_ => '9').mkString
+      val format3 = 0.until(input.length).map(i => i % 2 * 9).mkString
+      val format4 = 0.until(input.length + 1).map(_ => '0').mkString
+      val format5 = 0.until(input.length + 1).map(_ => '9').mkString
+      val format6 = 0.until(input.length + 1).map(i => i % 2 * 9).mkString
+      Seq(format1, format2, format3, format4, format5, format6).foreach { format =>
+        checkEvaluation(ToNumber(Literal(input), Literal(format)), Decimal(input))
+      }
+    }
+
+    // Test '.' and 'D'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.2"), Literal("999")),
+      "The input string '454.2' does not match the given number format: '999'")
+    Seq("999.9", "000.0", "99.99", "00.00", "0000.0", "9999.9", "00.000", "99.999")
+      .foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454.23"), Literal(format)),
+          s"The input string '454.23' does not match the given number format: '$format'")
+        val format2 = format.replace('.', 'D')
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454.23"), Literal(format2)),
+          s"The input string '454.23' does not match the given number format: '$format2'")
+    }
+
+    Seq(
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),
+      ("454.23", "000.00") -> Decimal(454.23),
+      ("454.23", "000D00") -> Decimal(454.23),
+      ("454.2", "000.00") -> Decimal(454.2),
+      ("454.2", "000D00") -> Decimal(454.2),
+      ("454.0", "000.0") -> Decimal(454),
+      ("454.0", "000D0") -> Decimal(454),
+      ("454.00", "000.00") -> Decimal(454),
+      ("454.00", "000D00") -> Decimal(454),
+      (".4542", ".0000") -> Decimal(0.4542),
+      (".4542", "D0000") -> Decimal(0.4542),
+      ("4542.", "0000.") -> Decimal(4542),
+      ("4542.", "0000D") -> Decimal(4542)
+    ).foreach { case ((str, format), expected) =>

Review comment:
       we can replace `.` with `D` in the look body, to avoid repeating the test cases manually.

##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,158 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+
+    Seq("454", "054", "54", "450").foreach { input =>
+      val invalidFormat1 = 0.until(input.length - 1).map(_ => '0').mkString
+      val invalidFormat2 = 0.until(input.length - 2).map(_ => '0').mkString
+      val invalidFormat3 = 0.until(input.length - 1).map(_ => '9').mkString
+      val invalidFormat4 = 0.until(input.length - 2).map(_ => '9').mkString
+      Seq(invalidFormat1, invalidFormat2, invalidFormat3, invalidFormat4)
+        .filter(_.nonEmpty).foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal(input), Literal(format)),
+          s"The input string '$input' does not match the given number format: '$format'")
+      }
+
+      val format1 = 0.until(input.length).map(_ => '0').mkString
+      val format2 = 0.until(input.length).map(_ => '9').mkString
+      val format3 = 0.until(input.length).map(i => i % 2 * 9).mkString
+      val format4 = 0.until(input.length + 1).map(_ => '0').mkString
+      val format5 = 0.until(input.length + 1).map(_ => '9').mkString
+      val format6 = 0.until(input.length + 1).map(i => i % 2 * 9).mkString
+      Seq(format1, format2, format3, format4, format5, format6).foreach { format =>
+        checkEvaluation(ToNumber(Literal(input), Literal(format)), Decimal(input))
+      }
+    }
+
+    // Test '.' and 'D'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.2"), Literal("999")),
+      "The input string '454.2' does not match the given number format: '999'")
+    Seq("999.9", "000.0", "99.99", "00.00", "0000.0", "9999.9", "00.000", "99.999")
+      .foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454.23"), Literal(format)),
+          s"The input string '454.23' does not match the given number format: '$format'")
+        val format2 = format.replace('.', 'D')
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454.23"), Literal(format2)),
+          s"The input string '454.23' does not match the given number format: '$format2'")
+    }
+
+    Seq(
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),
+      ("454.23", "000.00") -> Decimal(454.23),
+      ("454.23", "000D00") -> Decimal(454.23),
+      ("454.2", "000.00") -> Decimal(454.2),
+      ("454.2", "000D00") -> Decimal(454.2),
+      ("454.0", "000.0") -> Decimal(454),
+      ("454.0", "000D0") -> Decimal(454),
+      ("454.00", "000.00") -> Decimal(454),
+      ("454.00", "000D00") -> Decimal(454),
+      (".4542", ".0000") -> Decimal(0.4542),
+      (".4542", "D0000") -> Decimal(0.4542),
+      ("4542.", "0000.") -> Decimal(4542),
+      ("4542.", "0000D") -> Decimal(4542)
+    ).foreach { case ((str, format), expected) =>

Review comment:
       we can replace `.` with `D` in the loop body, to avoid repeating the test cases manually.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779595171



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0999|

Review comment:
       ```suggestion
   |**0**|digit position (will not be dropped, even if insignificant)|0000|
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779597016



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0999|
+|**.**|decimal point (only allowed once)|99.99|
+|**D**|decimal point (only allowed once)|99D99|
+|**,**|group (thousands) separator|9,999|
+|**G**|group (thousands) separator|9G999|
+|**-**|sign anchored to number|-9999|
+|**S**|sign anchored to number|S9999|
+|**$**|returns value with a leading dollar sign|$9999|
+
+Usage notes for numeric formatting:
+
+- 0 specifies a digit position that will always be printed, even if it contains a leading/trailing zero. 9 also specifies a digit position, but if it is a leading zero then it will be replaced by a space, while if it is a trailing zero and fill mode is specified then it will be deleted. (For to_number(), these two pattern characters are equivalent.)
+
+- The pattern characters S, D, and G represent the sign, decimal point, and thousands separator characters. The pattern characters period and comma represent those exact characters, with the meanings of decimal point and thousands separator.

Review comment:
       Although `to_char` is not implemented yet, can we use `to_char` as an example to demonstrate the differences between 0 and 9?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776908998



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)

Review comment:
       Yes




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776638161



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)
+         '.':  decimal point (only allowed once)
+         ',':  group (thousands) separator
+         'S':  sign anchored to number (uses locale)
+         'D':  decimal point (uses locale)

Review comment:
       is it true? We simply relace `D` with `.`, and I don't see we handle locale 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783190494



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -164,26 +192,86 @@ object NumberUtils {
     val bigDecimal = input.toJavaBigDecimal
     val decimalPlainStr = bigDecimal.toPlainString
     if (decimalPlainStr.length > transformedFormat.length) {
-      transformedFormat.replaceAll("0", "#")
+      transformedFormat.replaceAll("0", POUND_SIGN_STRING)
     } else {
-      val decimalFormat = new DecimalFormat(transformedFormat)
-      var resultStr = decimalFormat.format(bigDecimal)
+      val numberDecimalFormat = {
+        val decimalFormat = new DecimalFormat()
+        decimalFormat.setParseBigDecimal(true)
+        try {
+          decimalFormat.applyLocalizedPattern(transformedFormat)
+        } catch {
+          case _: IllegalArgumentException =>
+            throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
+        }
+        decimalFormat
+      }
+      var resultStr = numberDecimalFormat.format(bigDecimal)
       // Since we trimmed the comma at the beginning or end of number format in function
       // `normalize`, we restore the comma to the result here.
       // For example, if the specified number format is "99,999," or ",999,999", function
       // `normalize` normalize them to "##,###" or "###,###".
       // new DecimalFormat("##,###").parse(12454) and new DecimalFormat("###,###").parse(124546)
       // will return "12,454" and "124,546" respectively. So we add ',' at the end and head of
       // the result, then the final output are "12,454," or ",124,546".
-      if (numberFormat.last == commaSign || numberFormat.last == letterCommaSign) {
-        resultStr = resultStr + commaSign
+      if (numberFormat.last == COMMA_SIGN || numberFormat.last == COMMA_LETTER) {
+        resultStr = resultStr + COMMA_SIGN
       }
-      if (numberFormat.charAt(0) == commaSign || numberFormat.charAt(0) == letterCommaSign) {
-        resultStr = commaSign + resultStr
+      if (numberFormat.charAt(0) == COMMA_SIGN || numberFormat.charAt(0) == COMMA_LETTER) {
+        resultStr = COMMA_SIGN + resultStr
       }
 
       resultStr
     }
   }
 
+  class NumberFormatBuilder(originNumberFormat: String) extends Serializable {
+
+    protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+    private val transformedFormat = transform(normalizedNumberFormat)
+
+    private lazy val numberDecimalFormat = {
+      val decimalFormat = new DecimalFormat()

Review comment:
       can't we do `new DecimalFormat(transformedFormat)`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783658998



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -18,52 +18,76 @@
 package org.apache.spark.sql.catalyst.util
 
 import java.math.BigDecimal
-import java.text.{DecimalFormat, NumberFormat, ParsePosition}
+import java.text.{DecimalFormat, ParsePosition}
 import java.util.Locale
 
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
 import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
 import org.apache.spark.sql.types.Decimal
 import org.apache.spark.unsafe.types.UTF8String
 
 object NumberUtils {
 
-  private val pointSign = '.'
-  private val letterPointSign = 'D'
-  private val commaSign = ','
-  private val letterCommaSign = 'G'
-  private val minusSign = '-'
-  private val letterMinusSign = 'S'
-  private val dollarSign = '$'
+  private final val POINT_SIGN = '.'
+  private final val POINT_LETTER = 'D'
+  private final val COMMA_SIGN = ','
+  private final val COMMA_LETTER = 'G'
+  private final val MINUS_SIGN = '-'
+  private final val MINUS_LETTER = 'S'
+  private final val DOLLAR_SIGN = '$'
+  private final val NINE_DIGIT = '9'
+  private final val ZERO_DIGIT = '0'
+  private final val POUND_SIGN = '#'
 
-  private val commaSignStr = commaSign.toString
+  private final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  private final val POUND_SIGN_STRING = POUND_SIGN.toString
 
+  private final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+
+  /**
+   * DecimalFormat provides '#' and '0' as placeholder of digit, ',' as grouping separator,
+   * '.' as decimal separator, '-' as minus, '$' as dollar, but not '9', 'G', 'D', 'S'. So we need
+   * replace them show below:
+   * 1. '9' -> '#'
+   * 2. 'G' -> ','
+   * 3. 'D' -> '.'
+   * 4. 'S' -> '-'
+   *
+   * Note: When calling format, we must preserve the digits after decimal point, so the digits
+   * after decimal point should be replaced as '0'. For example: '999.9' will be normalized as
+   * '###.0' and '999.99' will be normalized as '###.00', so if the input is 454, the format
+   * output will be 454.0 and 454.00 respectively.
+   *
+   * @param format number format string
+   * @return normalized number format string
+   */
   private def normalize(format: String): String = {
-    var notFindDecimalPoint = true
+    var flag = true

Review comment:
       OK




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783688043



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,174 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("99")))
+    }
+
+    Seq(
+      ("454", "999") -> Decimal(454),
+      ("054", "999") -> Decimal(54),
+      ("54", "999") -> Decimal(54),
+      ("404", "999") -> Decimal(404),
+      ("450", "999") -> Decimal(450),
+      ("454", "9999") -> Decimal(454),
+      ("054", "9999") -> Decimal(54),
+      ("404", "9999") -> Decimal(404),
+      ("450", "9999") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("0")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("00")))
+    }
+
+    Seq(
+      ("454", "000") -> Decimal(454),
+      ("054", "000") -> Decimal(54),
+      ("54", "000") -> Decimal(54),
+      ("404", "000") -> Decimal(404),
+      ("450", "000") -> Decimal(450),
+      ("454", "0000") -> Decimal(454),
+      ("054", "0000") -> Decimal(54),
+      ("404", "0000") -> Decimal(404),
+      ("450", "0000") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    // Test '.' and 'D'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.2"), Literal("999")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.23"), Literal("999.9")))
+    }
+
+    Seq(
+      ("454.2", "999.9") -> Decimal(454.2),
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "999D9") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),
+      ("454.23", "999.99") -> Decimal(454.23),
+      ("454.23", "000.00") -> Decimal(454.23),
+      ("454.23", "999D99") -> Decimal(454.23),
+      ("454.23", "000D00") -> Decimal(454.23),
+      ("454.0", "999.9") -> Decimal(454),
+      ("454.0", "000.0") -> Decimal(454),
+      ("454.0", "999D9") -> Decimal(454),
+      ("454.0", "000D0") -> Decimal(454),
+      ("454.00", "999.99") -> Decimal(454),
+      ("454.00", "000.00") -> Decimal(454),
+      ("454.00", "999D99") -> Decimal(454),
+      ("454.00", "000D00") -> Decimal(454),
+      (".4542", ".9999") -> Decimal(0.4542),
+      (".4542", ".0000") -> Decimal(0.4542),
+      (".4542", "D9999") -> Decimal(0.4542),
+      (".4542", "D0000") -> Decimal(0.4542),
+      ("4542.", "9999.") -> Decimal(4542),
+      ("4542.", "0000.") -> Decimal(4542),
+      ("4542.", "9999D") -> Decimal(4542),
+      ("4542.", "0000D") -> Decimal(4542)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    Seq("999.9.9", "999D9D9", "999.9D9", "999D9.9").foreach { str =>
+      ToNumber(Literal("454.3.2"), Literal(str)).checkInputDataTypes() match {
+        case TypeCheckResult.TypeCheckFailure(msg) =>
+          assert(msg.contains(s"Multiple 'D' or '.' in '$str'"))
+      }
+    }
+
+    // Test ',' and 'G'
+    Seq(
+      ("12,454", "99,999") -> Decimal(12454),
+      ("12,454", "00,000") -> Decimal(12454),
+      ("12,454", "99G999") -> Decimal(12454),
+      ("12,454", "00G000") -> Decimal(12454),
+      ("12,454,367", "99,999,999") -> Decimal(12454367),
+      ("12,454,367", "00,000,000") -> Decimal(12454367),
+      ("12,454,367", "99G999G999") -> Decimal(12454367),
+      ("12,454,367", "00G000G000") -> Decimal(12454367),
+      ("12,454,", "99,999,") -> Decimal(12454),
+      ("12,454,", "00,000,") -> Decimal(12454),
+      ("12,454,", "99G999G") -> Decimal(12454),
+      ("12,454,", "00G000G") -> Decimal(12454),
+      (",454,367", ",999,999") -> Decimal(454367),

Review comment:
       `(",454,367", "999,999")` work good!  But `("123,456", "9G9")` will fail!. There exists a check the number of input digit must less or equal than the number of digit in format string.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r780091667



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,44 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0000|

Review comment:
       `Position for a digit; When formatting, it adds leading/trailing 0.`

##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,44 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0000|

Review comment:
       similar for `9`
   ```
   Position for a digit; When formatting, it adds a leading/trailing blank space.
   ```

##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,44 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0000|

Review comment:
       `Position for a digit; When formatting, it adds leading/trailing 0.`

##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,44 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0000|

Review comment:
       similar for `9`
   ```
   Position for a digit; When formatting, it adds a leading/trailing blank space.
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784010017



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatBuilder(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {
+    val decimalFormat = new DecimalFormat(transformedFormat)
+    decimalFormat.setParseBigDecimal(true)
+    decimalFormat
+  }
+
+  private val precision = getPrecision(normalizedNumberFormat)
+
+  private val scale = getScale(normalizedNumberFormat)
+
+  def parsedDecimalType: DecimalType = DecimalType(precision, scale)
+
+  def check(): TypeCheckResult = {
+    try {
+      check(normalizedNumberFormat, originNumberFormat)
+    } catch {
+      case e: AnalysisException => return TypeCheckResult.TypeCheckFailure(e.getMessage)
+    }
+    TypeCheckResult.TypeCheckSuccess
+  }
+
+  def parse(input: UTF8String): Decimal = {
+    parse(input, originNumberFormat, numberDecimalFormat, precision, scale)

Review comment:
       do we really need two `parse` methods?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784511626



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.sql.AnalysisException
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {

Review comment:
       Class org.apache.spark.sql.catalyst.util.NumberFormatter already exists




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784467824



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.sql.AnalysisException
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatter(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private lazy val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {
+    val decimalFormat = new DecimalFormat(transformedFormat)
+    decimalFormat.setParseBigDecimal(true)
+    decimalFormat
+  }
+
+  private lazy val precision = normalizedNumberFormat.filterNot(isSign).length
+
+  private lazy val scale = {
+    val formatSplits = normalizedNumberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
+    }
+  }
+
+  def parsedDecimalType: DecimalType = DecimalType(precision, scale)
+
+  /**
+   * DecimalFormat provides '#' and '0' as placeholder of digit, ',' as grouping separator,
+   * '.' as decimal separator, '-' as minus, '$' as dollar, but not '9', 'G', 'D', 'S'. So we need
+   * replace them show below:
+   * 1. '9' -> '#'
+   * 2. 'G' -> ','
+   * 3. 'D' -> '.'
+   * 4. 'S' -> '-'
+   *
+   * Note: When calling format, we must preserve the digits after decimal point, so the digits
+   * after decimal point should be replaced as '0'. For example: '999.9' will be normalized as
+   * '###.0' and '999.99' will be normalized as '###.00', so if the input is 454, the format
+   * output will be 454.0 and 454.00 respectively.
+   *
+   * @param format number format string
+   * @return normalized number format string
+   */
+  private def normalize(format: String): String = {
+    var notFindDecimalPoint = true
+    val normalizedFormat = format.toUpperCase(Locale.ROOT).map {
+      case NINE_DIGIT if notFindDecimalPoint => POUND_SIGN
+      case NINE_DIGIT if !notFindDecimalPoint => ZERO_DIGIT
+      case COMMA_LETTER => COMMA_SIGN
+      case POINT_LETTER | POINT_SIGN =>
+        notFindDecimalPoint = false
+        POINT_SIGN
+      case MINUS_LETTER => MINUS_SIGN
+      case other => other
+    }
+    // If the comma is at the beginning or end of number format, then DecimalFormat will be
+    // invalid. For example, "##,###," or ",###,###" for DecimalFormat is invalid, so we must use
+    // "##,###" or "###,###".
+    normalizedFormat.stripPrefix(COMMA_SIGN_STRING).stripSuffix(COMMA_SIGN_STRING)
+  }
+
+  private def isSign(c: Char): Boolean = {
+    SIGN_SET.contains(c)
+  }
+
+  private def transform(format: String): String = {
+    if (format.contains(MINUS_SIGN)) {
+      // For example: '#.######' represents a positive number,
+      // but '#.######;#.######-' represents a negative number.
+      val positiveFormatString = format.replaceAll("-", "")
+      s"$positiveFormatString;$format"
+    } else {
+      format
+    }
+  }
+
+  def check(): TypeCheckResult = {
+    def invalidSignPosition(c: Char): Boolean = {
+      val signIndex = normalizedNumberFormat.indexOf(c)
+      signIndex > 0 && signIndex < normalizedNumberFormat.length - 1
+    }
+
+    def multipleSignInNumberFormatError(message: String): String = {
+      s"Multiple $message in '$originNumberFormat'"

Review comment:
       to make the message clearer, I'd say `At most one $sign is allowed in the number format: $originNumberFormat`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r787790548



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,158 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+
+    Seq("454", "054", "54", "450").foreach { input =>
+      val invalidFormat1 = 0.until(input.length - 1).map(_ => '0').mkString
+      val invalidFormat2 = 0.until(input.length - 2).map(_ => '0').mkString
+      val invalidFormat3 = 0.until(input.length - 1).map(_ => '9').mkString
+      val invalidFormat4 = 0.until(input.length - 2).map(_ => '9').mkString
+      Seq(invalidFormat1, invalidFormat2, invalidFormat3, invalidFormat4)
+        .filter(_.nonEmpty).foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal(input), Literal(format)),
+          s"The input string '$input' does not match the given number format: '$format'")
+      }
+
+      val format1 = 0.until(input.length).map(_ => '0').mkString
+      val format2 = 0.until(input.length).map(_ => '9').mkString
+      val format3 = 0.until(input.length).map(i => i % 2 * 9).mkString
+      val format4 = 0.until(input.length + 1).map(_ => '0').mkString
+      val format5 = 0.until(input.length + 1).map(_ => '9').mkString
+      val format6 = 0.until(input.length + 1).map(i => i % 2 * 9).mkString
+      Seq(format1, format2, format3, format4, format5, format6).foreach { format =>
+        checkEvaluation(ToNumber(Literal(input), Literal(format)), Decimal(input))
+      }
+    }
+
+    // Test '.' and 'D'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.2"), Literal("999")),
+      "The input string '454.2' does not match the given number format: '999'")
+    Seq("999.9", "000.0", "99.99", "00.00", "0000.0", "9999.9", "00.000", "99.999")
+      .foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454.23"), Literal(format)),
+          s"The input string '454.23' does not match the given number format: '$format'")
+        val format2 = format.replace('.', 'D')
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454.23"), Literal(format2)),
+          s"The input string '454.23' does not match the given number format: '$format2'")
+    }
+
+    Seq(
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),
+      ("454.23", "000.00") -> Decimal(454.23),
+      ("454.23", "000D00") -> Decimal(454.23),
+      ("454.2", "000.00") -> Decimal(454.2),
+      ("454.2", "000D00") -> Decimal(454.2),
+      ("454.0", "000.0") -> Decimal(454),
+      ("454.0", "000D0") -> Decimal(454),
+      ("454.00", "000.00") -> Decimal(454),
+      ("454.00", "000D00") -> Decimal(454),
+      (".4542", ".0000") -> Decimal(0.4542),
+      (".4542", "D0000") -> Decimal(0.4542),
+      ("4542.", "0000.") -> Decimal(4542),
+      ("4542.", "0000D") -> Decimal(4542)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    Seq("999.9.9", "999D9D9", "999.9D9", "999D9.9").foreach { str =>
+      ToNumber(Literal("454.3.2"), Literal(str)).checkInputDataTypes() match {
+        case TypeCheckResult.TypeCheckFailure(msg) =>
+          assert(msg.contains(s"At most one 'D' or '.' is allowed in the number format: '$str'"))
+      }
+    }
+
+    // Test ',' and 'G'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("123,456"), Literal("9G9")),
+      "The input string '123,456' does not match the given number format: '9G9'")
+
+    Seq(
+      ("12,454", "99,999") -> Decimal(12454),

Review comment:
       ditto, no need to repeat the test cases manually, we can replace `0` with `9` in the loop body.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r787791917



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,158 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+
+    Seq("454", "054", "54", "450").foreach { input =>
+      val invalidFormat1 = 0.until(input.length - 1).map(_ => '0').mkString
+      val invalidFormat2 = 0.until(input.length - 2).map(_ => '0').mkString
+      val invalidFormat3 = 0.until(input.length - 1).map(_ => '9').mkString
+      val invalidFormat4 = 0.until(input.length - 2).map(_ => '9').mkString
+      Seq(invalidFormat1, invalidFormat2, invalidFormat3, invalidFormat4)
+        .filter(_.nonEmpty).foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal(input), Literal(format)),
+          s"The input string '$input' does not match the given number format: '$format'")
+      }
+
+      val format1 = 0.until(input.length).map(_ => '0').mkString
+      val format2 = 0.until(input.length).map(_ => '9').mkString
+      val format3 = 0.until(input.length).map(i => i % 2 * 9).mkString
+      val format4 = 0.until(input.length + 1).map(_ => '0').mkString
+      val format5 = 0.until(input.length + 1).map(_ => '9').mkString
+      val format6 = 0.until(input.length + 1).map(i => i % 2 * 9).mkString
+      Seq(format1, format2, format3, format4, format5, format6).foreach { format =>
+        checkEvaluation(ToNumber(Literal(input), Literal(format)), Decimal(input))
+      }
+    }
+
+    // Test '.' and 'D'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.2"), Literal("999")),
+      "The input string '454.2' does not match the given number format: '999'")
+    Seq("999.9", "000.0", "99.99", "00.00", "0000.0", "9999.9", "00.000", "99.999")
+      .foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454.23"), Literal(format)),
+          s"The input string '454.23' does not match the given number format: '$format'")
+        val format2 = format.replace('.', 'D')
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454.23"), Literal(format2)),
+          s"The input string '454.23' does not match the given number format: '$format2'")
+    }
+
+    Seq(
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),
+      ("454.23", "000.00") -> Decimal(454.23),
+      ("454.23", "000D00") -> Decimal(454.23),
+      ("454.2", "000.00") -> Decimal(454.2),
+      ("454.2", "000D00") -> Decimal(454.2),
+      ("454.0", "000.0") -> Decimal(454),
+      ("454.0", "000D0") -> Decimal(454),
+      ("454.00", "000.00") -> Decimal(454),
+      ("454.00", "000D00") -> Decimal(454),
+      (".4542", ".0000") -> Decimal(0.4542),
+      (".4542", "D0000") -> Decimal(0.4542),
+      ("4542.", "0000.") -> Decimal(4542),
+      ("4542.", "0000D") -> Decimal(4542)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    Seq("999.9.9", "999D9D9", "999.9D9", "999D9.9").foreach { str =>
+      ToNumber(Literal("454.3.2"), Literal(str)).checkInputDataTypes() match {
+        case TypeCheckResult.TypeCheckFailure(msg) =>
+          assert(msg.contains(s"At most one 'D' or '.' is allowed in the number format: '$str'"))
+      }
+    }
+
+    // Test ',' and 'G'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("123,456"), Literal("9G9")),

Review comment:
       can we also test if the number of `,` doesn't match? e.g.
   ```
   to_number('123,456', '999,999,999')
   to_number('123,456,789', '999,999')
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779999061



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0999|
+|**.**|decimal point (only allowed once)|99.99|
+|**D**|decimal point (only allowed once)|99D99|
+|**,**|group (thousands) separator|9,999|
+|**G**|group (thousands) separator|9G999|
+|**-**|sign anchored to number|-9999|
+|**S**|sign anchored to number|S9999|
+|**$**|returns value with a leading dollar sign|$9999|
+
+Usage notes for numeric formatting:
+
+- 0 specifies a digit position that will always be printed, even if it contains a leading/trailing zero. 9 also specifies a digit position, but if it is a leading zero then it will be replaced by a space, while if it is a trailing zero and fill mode is specified then it will be deleted. (For to_number(), these two pattern characters are equivalent.)

Review comment:
       OK




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783693547



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -164,26 +192,74 @@ object NumberUtils {
     val bigDecimal = input.toJavaBigDecimal
     val decimalPlainStr = bigDecimal.toPlainString
     if (decimalPlainStr.length > transformedFormat.length) {
-      transformedFormat.replaceAll("0", "#")
+      transformedFormat.replaceAll("0", POUND_SIGN_STRING)
     } else {
-      val decimalFormat = new DecimalFormat(transformedFormat)
-      var resultStr = decimalFormat.format(bigDecimal)
+      val numberDecimalFormat = {
+        val decimalFormat = new DecimalFormat()
+        decimalFormat.setParseBigDecimal(true)
+        try {
+          decimalFormat.applyLocalizedPattern(transformedFormat)
+        } catch {
+          case _: IllegalArgumentException =>
+            throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
+        }
+        decimalFormat
+      }
+      var resultStr = numberDecimalFormat.format(bigDecimal)
       // Since we trimmed the comma at the beginning or end of number format in function
       // `normalize`, we restore the comma to the result here.
       // For example, if the specified number format is "99,999," or ",999,999", function
       // `normalize` normalize them to "##,###" or "###,###".
       // new DecimalFormat("##,###").parse(12454) and new DecimalFormat("###,###").parse(124546)
       // will return "12,454" and "124,546" respectively. So we add ',' at the end and head of
       // the result, then the final output are "12,454," or ",124,546".
-      if (numberFormat.last == commaSign || numberFormat.last == letterCommaSign) {
-        resultStr = resultStr + commaSign
+      if (numberFormat.last == COMMA_SIGN || numberFormat.last == COMMA_LETTER) {
+        resultStr = resultStr + COMMA_SIGN
       }
-      if (numberFormat.charAt(0) == commaSign || numberFormat.charAt(0) == letterCommaSign) {
-        resultStr = commaSign + resultStr
+      if (numberFormat.charAt(0) == COMMA_SIGN || numberFormat.charAt(0) == COMMA_LETTER) {
+        resultStr = COMMA_SIGN + resultStr
       }
 
       resultStr
     }
   }
 
+  class NumberFormatBuilder(originNumberFormat: String) extends Serializable {

Review comment:
       seems we only need it to define some constants




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781687130



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -77,57 +101,61 @@ object NumberUtils {
       signIndex > 0 && signIndex < format.length - 1
     }
 
-    if (normalizedFormat.count(_ == pointSign) > 1) {
+    if (normalizedFormat.length == 0) {
+      throw QueryCompilationErrors.emptyNumberFormatError()
+    } else if (normalizedFormat.count(_ == POINT_SIGN) > 1) {
       throw QueryCompilationErrors.multipleSignInNumberFormatError(
-        s"'$letterPointSign' or '$pointSign'", numberFormat)
-    } else if (normalizedFormat.count(_ == minusSign) > 1) {
+        s"'$POINT_LETTER' or '$POINT_SIGN'", numberFormat)
+    } else if (normalizedFormat.count(_ == MINUS_SIGN) > 1) {
       throw QueryCompilationErrors.multipleSignInNumberFormatError(
-        s"'$letterMinusSign' or '$minusSign'", numberFormat)
-    } else if (normalizedFormat.count(_ == dollarSign) > 1) {
-      throw QueryCompilationErrors.multipleSignInNumberFormatError(s"'$dollarSign'", numberFormat)
-    } else if (invalidSignPosition(normalizedFormat, minusSign)) {
+        s"'$MINUS_LETTER' or '$MINUS_SIGN'", numberFormat)
+    } else if (normalizedFormat.count(_ == DOLLAR_SIGN) > 1) {
+      throw QueryCompilationErrors.multipleSignInNumberFormatError(s"'$DOLLAR_SIGN'", numberFormat)
+    } else if (invalidSignPosition(normalizedFormat, MINUS_SIGN)) {
       throw QueryCompilationErrors.nonFistOrLastCharInNumberFormatError(
-        s"'$letterMinusSign' or '$minusSign'", numberFormat)
-    } else if (invalidSignPosition(normalizedFormat, dollarSign)) {
+        s"'$MINUS_LETTER' or '$MINUS_SIGN'", numberFormat)
+    } else if (invalidSignPosition(normalizedFormat, DOLLAR_SIGN)) {
       throw QueryCompilationErrors.nonFistOrLastCharInNumberFormatError(
-        s"'$dollarSign'", numberFormat)
+        s"'$DOLLAR_SIGN'", numberFormat)
+    }
+  }
+
+  private def getPrecision(numberFormat: String): Int =
+    numberFormat.filterNot(isSign).length
+
+  private def getScale(numberFormat: String): Int = {
+    val formatSplits = numberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
     }
   }
 
   /**
    * Convert string to numeric based on the given number format.
    * The format can consist of the following characters:
-   * '9':  digit position (can be dropped if insignificant)
-   * '0':  digit position (will not be dropped, even if insignificant)
-   * '.':  decimal point (only allowed once)
-   * ',':  group (thousands) separator
-   * 'S':  sign anchored to number (uses locale)
-   * 'D':  decimal point (uses locale)
-   * 'G':  group separator (uses locale)
-   * '$':  specifies that the input value has a leading $ (Dollar) sign.
+   * '0' or '9': digit position
+   * '.' or 'D': decimal point (only allowed once)
+   * ',' or 'G': group (thousands) separator
+   * '-' or 'S': sign anchored to number
+   * '$': returns value with a leading dollar sign
    *
    * @param input the string need to converted
    * @param numberFormat the given number format
+   * @param normalizedNumberFormat normalized number format
+   * @param precision decimal precision
+   * @param scale decimal scale
    * @return decimal obtained from string parsing
    */
-  def parse(input: UTF8String, numberFormat: String): Decimal = {
-    val normalizedFormat = normalize(numberFormat)
-    check(normalizedFormat, numberFormat)
-
-    val precision = normalizedFormat.filterNot(isSign).length
-    val formatSplits = normalizedFormat.split(pointSign)
-    val scale = if (formatSplits.length == 1) {
-      0
-    } else {
-      formatSplits(1).filterNot(isSign).length
-    }
-    val transformedFormat = transform(normalizedFormat)
-    val numberFormatInstance = NumberFormat.getInstance()
-    val numberDecimalFormat = numberFormatInstance.asInstanceOf[DecimalFormat]
-    numberDecimalFormat.setParseBigDecimal(true)
-    numberDecimalFormat.applyPattern(transformedFormat)
+  private def parse(
+      input: UTF8String,
+      numberFormat: String,
+      normalizedNumberFormat: String,
+      precision: Int,
+      scale: Int): Decimal = {
     val inputStr = input.toString.trim
-    val inputSplits = inputStr.split(pointSign)
+    val inputSplits = inputStr.split(POINT_SIGN)

Review comment:
       throws Exception.
   https://github.com/apache/spark/blob/518bdad0504baa4d9af64126bd66e021959f4846/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala#L107




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783643127



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,174 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("99")))
+    }
+
+    Seq(
+      ("454", "999") -> Decimal(454),
+      ("054", "999") -> Decimal(54),
+      ("54", "999") -> Decimal(54),
+      ("404", "999") -> Decimal(404),
+      ("450", "999") -> Decimal(450),
+      ("454", "9999") -> Decimal(454),
+      ("054", "9999") -> Decimal(54),
+      ("404", "9999") -> Decimal(404),
+      ("450", "9999") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("0")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("00")))
+    }
+
+    Seq(
+      ("454", "000") -> Decimal(454),
+      ("054", "000") -> Decimal(54),
+      ("54", "000") -> Decimal(54),
+      ("404", "000") -> Decimal(404),
+      ("450", "000") -> Decimal(450),
+      ("454", "0000") -> Decimal(454),
+      ("054", "0000") -> Decimal(54),
+      ("404", "0000") -> Decimal(404),
+      ("450", "0000") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    // Test '.' and 'D'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.2"), Literal("999")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.23"), Literal("999.9")))
+    }
+
+    Seq(
+      ("454.2", "999.9") -> Decimal(454.2),

Review comment:
       some suggestions for the tests:
   1. We know that some pattern chars are identical after normalization, and we don't need to test it everywhere. Just put a few tests to verify it.
   2. We should test more factors, e.g. the number of `0` or `0` is equal to, smaller than, or larger than the actual number of digits in the value.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781185971



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -136,21 +164,32 @@ object NumberUtils {
       inputSplits(1).filterNot(isSign).length > scale) {
       throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
     }
+
+    val transformedFormat = transform(normalizedNumberFormat)
+    val numberFormatInstance = NumberFormat.getNumberInstance(Locale.ROOT)
+    assert(numberFormatInstance.isInstanceOf[DecimalFormat])
+    val numberDecimalFormat = numberFormatInstance.asInstanceOf[DecimalFormat]

Review comment:
       It's super inefficient to create a `NumberFormat` instance for every input row.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781181319



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -77,57 +101,61 @@ object NumberUtils {
       signIndex > 0 && signIndex < format.length - 1
     }
 
-    if (normalizedFormat.count(_ == pointSign) > 1) {
+    if (normalizedFormat.length == 0) {
+      throw QueryCompilationErrors.emptyNumberFormatError()
+    } else if (normalizedFormat.count(_ == POINT_SIGN) > 1) {
       throw QueryCompilationErrors.multipleSignInNumberFormatError(
-        s"'$letterPointSign' or '$pointSign'", numberFormat)
-    } else if (normalizedFormat.count(_ == minusSign) > 1) {
+        s"'$POINT_LETTER' or '$POINT_SIGN'", numberFormat)
+    } else if (normalizedFormat.count(_ == MINUS_SIGN) > 1) {
       throw QueryCompilationErrors.multipleSignInNumberFormatError(
-        s"'$letterMinusSign' or '$minusSign'", numberFormat)
-    } else if (normalizedFormat.count(_ == dollarSign) > 1) {
-      throw QueryCompilationErrors.multipleSignInNumberFormatError(s"'$dollarSign'", numberFormat)
-    } else if (invalidSignPosition(normalizedFormat, minusSign)) {
+        s"'$MINUS_LETTER' or '$MINUS_SIGN'", numberFormat)
+    } else if (normalizedFormat.count(_ == DOLLAR_SIGN) > 1) {
+      throw QueryCompilationErrors.multipleSignInNumberFormatError(s"'$DOLLAR_SIGN'", numberFormat)
+    } else if (invalidSignPosition(normalizedFormat, MINUS_SIGN)) {
       throw QueryCompilationErrors.nonFistOrLastCharInNumberFormatError(
-        s"'$letterMinusSign' or '$minusSign'", numberFormat)
-    } else if (invalidSignPosition(normalizedFormat, dollarSign)) {
+        s"'$MINUS_LETTER' or '$MINUS_SIGN'", numberFormat)
+    } else if (invalidSignPosition(normalizedFormat, DOLLAR_SIGN)) {
       throw QueryCompilationErrors.nonFistOrLastCharInNumberFormatError(
-        s"'$dollarSign'", numberFormat)
+        s"'$DOLLAR_SIGN'", numberFormat)
+    }
+  }
+
+  private def getPrecision(numberFormat: String): Int =
+    numberFormat.filterNot(isSign).length
+
+  private def getScale(numberFormat: String): Int = {
+    val formatSplits = numberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
     }
   }
 
   /**
    * Convert string to numeric based on the given number format.
    * The format can consist of the following characters:
-   * '9':  digit position (can be dropped if insignificant)
-   * '0':  digit position (will not be dropped, even if insignificant)
-   * '.':  decimal point (only allowed once)
-   * ',':  group (thousands) separator
-   * 'S':  sign anchored to number (uses locale)
-   * 'D':  decimal point (uses locale)
-   * 'G':  group separator (uses locale)
-   * '$':  specifies that the input value has a leading $ (Dollar) sign.
+   * '0' or '9': digit position
+   * '.' or 'D': decimal point (only allowed once)
+   * ',' or 'G': group (thousands) separator
+   * '-' or 'S': sign anchored to number
+   * '$': returns value with a leading dollar sign

Review comment:
       `returns`? This is `def parse`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786115746



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,179 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+
+    Seq("454", "054", "54", "450").foreach { input =>
+      val invalidFormat1 = 0.until(input.length - 1).map(_ => '0').mkString
+      val invalidFormat2 = 0.until(input.length - 2).map(_ => '0').mkString
+      val invalidFormat3 = 0.until(input.length - 1).map(_ => '9').mkString
+      val invalidFormat4 = 0.until(input.length - 2).map(_ => '9').mkString
+      Seq(invalidFormat1, invalidFormat2, invalidFormat3, invalidFormat4)
+        .filter(_.nonEmpty).foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454"), Literal(format)), s"Format '$format' used for" +
+            " parsing string to number or formatting number to string is invalid")
+      }
+
+      val format1 = 0.until(input.length).map(_ => '0').mkString
+      val format2 = 0.until(input.length).map(_ => '9').mkString
+      val format3 = 0.until(input.length + 1).map(_ => '0').mkString
+      val format4 = 0.until(input.length + 1).map(_ => '9').mkString
+      val format5 = 0.until(input.length + 2).map(_ => '0').mkString

Review comment:
       Can we test the mixed case as I proposed before? `val format3 = 0.until(input.length + 2).map(i => i % 2).mkString`
   
   It's usually not very useful to go deeper into one factor in the test. To test format string that is longer than the value, the `+ 1` case should be sufficient, we don't need to test `+ 2`.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r787795046



##########
File path: sql/core/src/test/resources/sql-tests/inputs/string-functions.sql
##########
@@ -124,4 +124,22 @@ SELECT endswith('Spark SQL', 'QL');
 SELECT endswith('Spark SQL', 'Spa');
 SELECT endswith(null, 'Spark');
 SELECT endswith('Spark', null);
-SELECT endswith(null, null);
\ No newline at end of file
+SELECT endswith(null, null);
+
+-- to_number
+select to_number('454', '999');
+select to_number('454', '000');

Review comment:
       we can use `0` alone, instead of repeating the tests and use `9`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776640261



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)
+         '.':  decimal point (only allowed once)
+         ',':  group (thousands) separator
+         'S':  sign anchored to number (uses locale)
+         'D':  decimal point (uses locale)
+         'G':  group separator (uses locale)
+         '$':  specifies that the input value has a leading $ (Dollar) sign.
+  """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_('454', '999');
+       454
+      > SELECT _FUNC_('454.00', '000D00');
+       454.00
+      > SELECT _FUNC_('12,454', '99G999');
+       12454
+      > SELECT _FUNC_('$78.12', '$99.99');
+       78.12
+      > SELECT _FUNC_('12,454.8-', '99G999D9S');
+       -12454.8
+  """,
+  since = "3.3.0",
+  group = "string_funcs")
+case class ToNumber(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant {
+
+  private lazy val numberFormat = right.eval().toString.toUpperCase(Locale.ROOT)
+  private lazy val builder = new NumberFormatBuilder(numberFormat)
+  private lazy val (precision, scale) = builder.parsePrecisionAndScale()
+
+  override def dataType: DataType = DecimalType(precision, scale)
+
+  override def inputTypes: Seq[DataType] = Seq(StringType, StringType)
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val inputTypeCheck = super.checkInputDataTypes()
+    if (inputTypeCheck.isSuccess) {
+      if (right.foldable) {
+        if (builder.normalizedNumberFormat.length == 0) {
+          TypeCheckResult.TypeCheckFailure(s"Format expression cannot be empty")
+        } else {
+          builder.check()
+        }
+      } else {
+        TypeCheckResult.TypeCheckFailure(s"Format expression must be foldable, but got $right")
+      }
+    } else {
+      inputTypeCheck
+    }
+  }
+
+  override def prettyName: String = "to_number"
+
+  override def nullSafeEval(string: Any, format: Any): Any = {
+    val input = string.asInstanceOf[UTF8String]
+    builder.parse(input)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+    val builderClass = classOf[NumberFormatBuilder].getName
+    val builder = ctx.freshName("builder")
+    val eval = left.genCode(ctx)
+    ev.copy(code = code"""
+      ${eval.code}
+      boolean ${ev.isNull} = ${eval.isNull};
+      ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
+      if (!${ev.isNull}) {
+        $builderClass $builder = new $builderClass("$numberFormat");

Review comment:
       It's pretty risky to embed user input in the generated code, as people can inject malicious code.
   
   We should do `ctx.addReferenceObj(builder)`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776928591



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)

Review comment:
       Then why do we document them differently here?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779597316



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0999|
+|**.**|decimal point (only allowed once)|99.99|
+|**D**|decimal point (only allowed once)|99D99|
+|**,**|group (thousands) separator|9,999|
+|**G**|group (thousands) separator|9G999|
+|**-**|sign anchored to number|-9999|
+|**S**|sign anchored to number|S9999|
+|**$**|returns value with a leading dollar sign|$9999|
+
+Usage notes for numeric formatting:
+
+- 0 specifies a digit position that will always be printed, even if it contains a leading/trailing zero. 9 also specifies a digit position, but if it is a leading zero then it will be replaced by a space, while if it is a trailing zero and fill mode is specified then it will be deleted. (For to_number(), these two pattern characters are equivalent.)

Review comment:
       Although `to_char` is not implemented yet, can we use `to_char` as an example to demonstrate the differences between 0 and 9?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779592689



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|

Review comment:
       ```suggestion
   |**9**|Position for a digit; leading/trailing zeros are trimmed.|9999|
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r780091667



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,44 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0000|

Review comment:
       `Position for a digit; When formatting, it adds leading/trailing 0.`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r788554017



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberFormatterSuite.scala
##########
@@ -19,76 +19,75 @@ package org.apache.spark.sql.catalyst.util
 
 import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.util.NumberUtils.{format, parse}
 import org.apache.spark.sql.types.Decimal
 import org.apache.spark.unsafe.types.UTF8String
 
-class NumberUtilsSuite extends SparkFunSuite {
+class NumberFormatterSuite extends SparkFunSuite {
 
-  private def failParseWithInvalidInput(
-      input: UTF8String, numberFormat: String, errorMsg: String): Unit = {
-    val e = intercept[IllegalArgumentException](parse(input, numberFormat))
+  private def invalidNumberFormat(numberFormat: String, errorMsg: String): Unit = {
+    val testNumberFormatter = new TestNumberFormatter(numberFormat)

Review comment:
       do we really need this `TestNumberFormatter`? We can simply call `check`, expect it to return `TypeCheckFailure`, and check the error message.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779593483



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|

Review comment:
       I noticed that snowflake does not simply trim the leading/trailing zeros, but replace with spaces. Is it a common behavior in other databases?

##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|

Review comment:
       I noticed that snowflake does not simply trim the leading/trailing zeros, but replace them with spaces. Is it a common behavior in other databases?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r777835634



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)
+         '.':  decimal point (only allowed once)
+         ',':  group (thousands) separator
+         'S':  sign anchored to number (uses locale)
+         'D':  decimal point (uses locale)

Review comment:
       are D, S, G commonly supported in other databases? If not, we can remove them as we don't support locale.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r780001554



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|

Review comment:
       Yes. `to_char` will replace leading 9 with spaces.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781172352



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -21,49 +21,73 @@ import java.math.BigDecimal
 import java.text.{DecimalFormat, NumberFormat, ParsePosition}
 import java.util.Locale
 
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
 import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
 import org.apache.spark.sql.types.Decimal
 import org.apache.spark.unsafe.types.UTF8String
 
 object NumberUtils {
 
-  private val pointSign = '.'
-  private val letterPointSign = 'D'
-  private val commaSign = ','
-  private val letterCommaSign = 'G'
-  private val minusSign = '-'
-  private val letterMinusSign = 'S'
-  private val dollarSign = '$'
+  private final val POINT_SIGN = '.'
+  private final val POINT_LETTER = 'D'
+  private final val COMMA_SIGN = ','
+  private final val COMMA_LETTER = 'G'
+  private final val MINUS_SIGN = '-'
+  private final val MINUS_LETTER = 'S'
+  private final val DOLLAR_SIGN = '$'
+  private final val NINE_DIGIT = '9'
+  private final val ZERO_DIGIT = '0'
+  private final val POUND_SIGN = '#'
 
-  private val commaSignStr = commaSign.toString
+  private final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  private final val POUND_SIGN_STRING = POUND_SIGN.toString
 
+  private final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+
+  /**
+   * DecimalFormat provides '#' and '0' as placeholder of digit, ',' as grouping separator,
+   * '.' as decimal separator, '-' as minus, '$' as dollar, but '9', 'G', 'D', 'S'. So we need
+   * replace them show below:
+   * 1. '9' -> '#'
+   * 2. 'G' -> ','
+   * 3. 'D' -> '.'
+   * 4. 'S' -> '-'
+   *
+   * Note: When calling format, we must preserve the digits after decimal point, so the digits

Review comment:
       I'm not very sure about this behavior, but let's leave the discussion when adding `to_char`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r782794998



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -77,80 +107,91 @@ object NumberUtils {
       signIndex > 0 && signIndex < format.length - 1
     }
 
-    if (normalizedFormat.count(_ == pointSign) > 1) {
+    if (normalizedFormat.length == 0) {
+      throw QueryCompilationErrors.emptyNumberFormatError()
+    } else if (normalizedFormat.count(_ == POINT_SIGN) > 1) {
       throw QueryCompilationErrors.multipleSignInNumberFormatError(
-        s"'$letterPointSign' or '$pointSign'", numberFormat)
-    } else if (normalizedFormat.count(_ == minusSign) > 1) {
+        s"'$POINT_LETTER' or '$POINT_SIGN'", numberFormat)
+    } else if (normalizedFormat.count(_ == MINUS_SIGN) > 1) {
       throw QueryCompilationErrors.multipleSignInNumberFormatError(
-        s"'$letterMinusSign' or '$minusSign'", numberFormat)
-    } else if (normalizedFormat.count(_ == dollarSign) > 1) {
-      throw QueryCompilationErrors.multipleSignInNumberFormatError(s"'$dollarSign'", numberFormat)
-    } else if (invalidSignPosition(normalizedFormat, minusSign)) {
+        s"'$MINUS_LETTER' or '$MINUS_SIGN'", numberFormat)
+    } else if (normalizedFormat.count(_ == DOLLAR_SIGN) > 1) {
+      throw QueryCompilationErrors.multipleSignInNumberFormatError(s"'$DOLLAR_SIGN'", numberFormat)
+    } else if (invalidSignPosition(normalizedFormat, MINUS_SIGN)) {
       throw QueryCompilationErrors.nonFistOrLastCharInNumberFormatError(
-        s"'$letterMinusSign' or '$minusSign'", numberFormat)
-    } else if (invalidSignPosition(normalizedFormat, dollarSign)) {
+        s"'$MINUS_LETTER' or '$MINUS_SIGN'", numberFormat)
+    } else if (invalidSignPosition(normalizedFormat, DOLLAR_SIGN)) {
       throw QueryCompilationErrors.nonFistOrLastCharInNumberFormatError(
-        s"'$dollarSign'", numberFormat)
+        s"'$DOLLAR_SIGN'", numberFormat)
+    }
+  }
+
+  private def getPrecision(numberFormat: String): Int =
+    numberFormat.filterNot(isSign).length
+
+  private def getScale(numberFormat: String): Int = {
+    val formatSplits = numberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
     }
   }
 
   /**
    * Convert string to numeric based on the given number format.
    * The format can consist of the following characters:
-   * '9':  digit position (can be dropped if insignificant)
-   * '0':  digit position (will not be dropped, even if insignificant)
-   * '.':  decimal point (only allowed once)
-   * ',':  group (thousands) separator
-   * 'S':  sign anchored to number (uses locale)
-   * 'D':  decimal point (uses locale)
-   * 'G':  group separator (uses locale)
-   * '$':  specifies that the input value has a leading $ (Dollar) sign.
+   * '0' or '9': digit position
+   * '.' or 'D': decimal point (only allowed once)
+   * ',' or 'G': group (thousands) separator
+   * '-' or 'S': sign anchored to number (only allowed once)
+   * '$': value with a leading dollar sign (only allowed once)
    *
    * @param input the string need to converted
-   * @param numberFormat the given number format
+   * @param originNumberFormat the origin number format
+   * @param normalizedNumberFormat normalized number format
+   * @param precision decimal precision
+   * @param scale decimal scale
    * @return decimal obtained from string parsing
    */
-  def parse(input: UTF8String, numberFormat: String): Decimal = {
-    val normalizedFormat = normalize(numberFormat)
-    check(normalizedFormat, numberFormat)
-
-    val precision = normalizedFormat.filterNot(isSign).length
-    val formatSplits = normalizedFormat.split(pointSign)
-    val scale = if (formatSplits.length == 1) {
-      0
-    } else {
-      formatSplits(1).filterNot(isSign).length
-    }
-    val transformedFormat = transform(normalizedFormat)
-    val numberFormatInstance = NumberFormat.getInstance()
-    val numberDecimalFormat = numberFormatInstance.asInstanceOf[DecimalFormat]
-    numberDecimalFormat.setParseBigDecimal(true)
-    numberDecimalFormat.applyPattern(transformedFormat)
+  private def parse(
+      input: UTF8String,
+      originNumberFormat: String,
+      normalizedNumberFormat: String,
+      precision: Int,
+      scale: Int): Decimal = {
     val inputStr = input.toString.trim
-    val inputSplits = inputStr.split(pointSign)
+    val inputSplits = inputStr.split(POINT_SIGN)
     if (inputSplits.length == 1) {
       if (inputStr.filterNot(isSign).length > precision - scale) {
-        throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
+        throw QueryExecutionErrors.invalidNumberFormatError(originNumberFormat)
       }
     } else if (inputSplits(0).filterNot(isSign).length > precision - scale ||
       inputSplits(1).filterNot(isSign).length > scale) {
-      throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
+      throw QueryExecutionErrors.invalidNumberFormatError(originNumberFormat)
+    }
+
+    val transformedFormat = transform(normalizedNumberFormat)
+    try {
+      numberDecimalFormat.applyLocalizedPattern(transformedFormat)

Review comment:
       do we need to do this for every row?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783639168



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '0' or '9':  digit position
+         '.' or 'D':  decimal point (only allowed once)
+         ',' or 'G':  group (thousands) separator
+         '-' or 'S':  sign anchored to number (only allowed once)
+         '$':  value with a leading dollar sign (only allowed once)
+  """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_('454', '999');
+       454
+      > SELECT _FUNC_('454.00', '000D00');
+       454.00
+      > SELECT _FUNC_('12,454', '99G999');
+       12454
+      > SELECT _FUNC_('$78.12', '$99.99');
+       78.12
+      > SELECT _FUNC_('12,454.8-', '99G999D9S');
+       -12454.8
+  """,
+  since = "3.3.0",
+  group = "string_funcs")
+case class ToNumber(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant {
+
+  private lazy val numberFormat = right.eval().toString.toUpperCase(Locale.ROOT)
+  private lazy val numberFormatBuilder = new NumberFormatBuilder(numberFormat)
+  private lazy val (precision, scale) = numberFormatBuilder.parsePrecisionAndScale()

Review comment:
       nit: it's simpler to have a `def parsedDecimalType: DecimalType` and use it here.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776667842



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)
+         '.':  decimal point (only allowed once)
+         ',':  group (thousands) separator
+         'S':  sign anchored to number (uses locale)
+         'D':  decimal point (uses locale)
+         'G':  group separator (uses locale)
+         '$':  specifies that the input value has a leading $ (Dollar) sign.
+  """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_('454', '999');
+       454
+      > SELECT _FUNC_('454.00', '000D00');
+       454.00
+      > SELECT _FUNC_('12,454', '99G999');
+       12454
+      > SELECT _FUNC_('$78.12', '$99.99');
+       78.12
+      > SELECT _FUNC_('12,454.8-', '99G999D9S');
+       -12454.8
+  """,
+  since = "3.3.0",
+  group = "string_funcs")
+case class ToNumber(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant {
+
+  private lazy val numberFormat = right.eval().toString.toUpperCase(Locale.ROOT)
+  private lazy val builder = new NumberFormatBuilder(numberFormat)
+  private lazy val (precision, scale) = builder.parsePrecisionAndScale()
+
+  override def dataType: DataType = DecimalType(precision, scale)
+
+  override def inputTypes: Seq[DataType] = Seq(StringType, StringType)
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val inputTypeCheck = super.checkInputDataTypes()
+    if (inputTypeCheck.isSuccess) {
+      if (right.foldable) {
+        if (builder.normalizedNumberFormat.length == 0) {
+          TypeCheckResult.TypeCheckFailure(s"Format expression cannot be empty")

Review comment:
       OK




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776673212



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)
+         '.':  decimal point (only allowed once)
+         ',':  group (thousands) separator
+         'S':  sign anchored to number (uses locale)
+         'D':  decimal point (uses locale)

Review comment:
       Please see https://github.com/apache/spark/blob/5cfb5dfde5e566a4380def46324034604012af32/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala#L172




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776672696



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)

Review comment:
       I updated some comment. Please review again.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784469883



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.sql.AnalysisException
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {

Review comment:
       Since these constants are only used by `NumberFormatter`, I think it's OK to name this object `NumberFormatter`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r788313185



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberFormatterSuite.scala
##########
@@ -19,76 +19,75 @@ package org.apache.spark.sql.catalyst.util
 
 import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.util.NumberUtils.{format, parse}
 import org.apache.spark.sql.types.Decimal
 import org.apache.spark.unsafe.types.UTF8String
 
-class NumberUtilsSuite extends SparkFunSuite {
+class NumberFormatterSuite extends SparkFunSuite {

Review comment:
       Just follows `StringUtils` with `StringUtilsSuite`. In fact, `RegexpExpressionsSuite.scala` test almost everything too.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783191609



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -164,26 +192,86 @@ object NumberUtils {
     val bigDecimal = input.toJavaBigDecimal
     val decimalPlainStr = bigDecimal.toPlainString
     if (decimalPlainStr.length > transformedFormat.length) {
-      transformedFormat.replaceAll("0", "#")
+      transformedFormat.replaceAll("0", POUND_SIGN_STRING)
     } else {
-      val decimalFormat = new DecimalFormat(transformedFormat)
-      var resultStr = decimalFormat.format(bigDecimal)
+      val numberDecimalFormat = {
+        val decimalFormat = new DecimalFormat()
+        decimalFormat.setParseBigDecimal(true)
+        try {
+          decimalFormat.applyLocalizedPattern(transformedFormat)
+        } catch {
+          case _: IllegalArgumentException =>
+            throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
+        }
+        decimalFormat
+      }
+      var resultStr = numberDecimalFormat.format(bigDecimal)
       // Since we trimmed the comma at the beginning or end of number format in function
       // `normalize`, we restore the comma to the result here.
       // For example, if the specified number format is "99,999," or ",999,999", function
       // `normalize` normalize them to "##,###" or "###,###".
       // new DecimalFormat("##,###").parse(12454) and new DecimalFormat("###,###").parse(124546)
       // will return "12,454" and "124,546" respectively. So we add ',' at the end and head of
       // the result, then the final output are "12,454," or ",124,546".
-      if (numberFormat.last == commaSign || numberFormat.last == letterCommaSign) {
-        resultStr = resultStr + commaSign
+      if (numberFormat.last == COMMA_SIGN || numberFormat.last == COMMA_LETTER) {
+        resultStr = resultStr + COMMA_SIGN
       }
-      if (numberFormat.charAt(0) == commaSign || numberFormat.charAt(0) == letterCommaSign) {
-        resultStr = commaSign + resultStr
+      if (numberFormat.charAt(0) == COMMA_SIGN || numberFormat.charAt(0) == COMMA_LETTER) {
+        resultStr = COMMA_SIGN + resultStr
       }
 
       resultStr
     }
   }
 
+  class NumberFormatBuilder(originNumberFormat: String) extends Serializable {
+
+    protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+    private val transformedFormat = transform(normalizedNumberFormat)
+
+    private lazy val numberDecimalFormat = {
+      val decimalFormat = new DecimalFormat()
+      decimalFormat.setParseBigDecimal(true)
+      try {
+        decimalFormat.applyLocalizedPattern(transformedFormat)
+      } catch {
+        case _: IllegalArgumentException =>
+          throw QueryExecutionErrors.invalidNumberFormatError(originNumberFormat)
+      }
+      decimalFormat
+    }
+
+    private val precision = getPrecision(normalizedNumberFormat)
+
+    private val scale = getScale(normalizedNumberFormat)
+
+    private var unChecked = true
+
+    def parsePrecisionAndScale(): (Int, Int) = (precision, scale)
+
+    def check(): TypeCheckResult = {
+      try {
+        NumberUtils.check(normalizedNumberFormat, originNumberFormat)
+      } catch {
+        case e: AnalysisException => return TypeCheckResult.TypeCheckFailure(e.getMessage)
+      }
+      unChecked = false
+      TypeCheckResult.TypeCheckSuccess
+    }
+
+    def parse(input: UTF8String): Decimal = {
+      if (unChecked) {
+        check()

Review comment:
       I'm confused. This is a static type check, why do you do it at runtime when parsing the first input?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783189547



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -164,26 +192,86 @@ object NumberUtils {
     val bigDecimal = input.toJavaBigDecimal
     val decimalPlainStr = bigDecimal.toPlainString
     if (decimalPlainStr.length > transformedFormat.length) {
-      transformedFormat.replaceAll("0", "#")
+      transformedFormat.replaceAll("0", POUND_SIGN_STRING)
     } else {
-      val decimalFormat = new DecimalFormat(transformedFormat)
-      var resultStr = decimalFormat.format(bigDecimal)
+      val numberDecimalFormat = {
+        val decimalFormat = new DecimalFormat()

Review comment:
       can we do `new DecimalFormat(transformedFormat)`?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784011365



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatBuilder(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {
+    val decimalFormat = new DecimalFormat(transformedFormat)
+    decimalFormat.setParseBigDecimal(true)
+    decimalFormat
+  }
+
+  private val precision = getPrecision(normalizedNumberFormat)
+
+  private val scale = getScale(normalizedNumberFormat)
+
+  def parsedDecimalType: DecimalType = DecimalType(precision, scale)
+
+  def check(): TypeCheckResult = {
+    try {
+      check(normalizedNumberFormat, originNumberFormat)
+    } catch {
+      case e: AnalysisException => return TypeCheckResult.TypeCheckFailure(e.getMessage)
+    }
+    TypeCheckResult.TypeCheckSuccess
+  }
+
+  def parse(input: UTF8String): Decimal = {
+    parse(input, originNumberFormat, numberDecimalFormat, precision, scale)
+  }
+
+  /**
+   * DecimalFormat provides '#' and '0' as placeholder of digit, ',' as grouping separator,
+   * '.' as decimal separator, '-' as minus, '$' as dollar, but not '9', 'G', 'D', 'S'. So we need
+   * replace them show below:
+   * 1. '9' -> '#'
+   * 2. 'G' -> ','
+   * 3. 'D' -> '.'
+   * 4. 'S' -> '-'
+   *
+   * Note: When calling format, we must preserve the digits after decimal point, so the digits
+   * after decimal point should be replaced as '0'. For example: '999.9' will be normalized as
+   * '###.0' and '999.99' will be normalized as '###.00', so if the input is 454, the format
+   * output will be 454.0 and 454.00 respectively.
+   *
+   * @param format number format string
+   * @return normalized number format string
+   */
+  private def normalize(format: String): String = {
+    var notFindDecimalPoint = true
+    val normalizedFormat = format.toUpperCase(Locale.ROOT).map {
+      case NINE_DIGIT if notFindDecimalPoint => POUND_SIGN
+      case NINE_DIGIT if !notFindDecimalPoint => ZERO_DIGIT
+      case COMMA_LETTER => COMMA_SIGN
+      case POINT_LETTER | POINT_SIGN =>
+        notFindDecimalPoint = false
+        POINT_SIGN
+      case MINUS_LETTER => MINUS_SIGN
+      case other => other
+    }
+    // If the comma is at the beginning or end of number format, then DecimalFormat will be
+    // invalid. For example, "##,###," or ",###,###" for DecimalFormat is invalid, so we must use
+    // "##,###" or "###,###".
+    normalizedFormat.stripPrefix(COMMA_SIGN_STRING).stripSuffix(COMMA_SIGN_STRING)
+  }
+
+  private def isSign(c: Char): Boolean = {
+    SIGN_SET.contains(c)
+  }
+
+  private def transform(format: String): String = {
+    if (format.contains(MINUS_SIGN)) {
+      // For example: '#.######' represents a positive number,
+      // but '#.######;#.######-' represents a negative number.
+      val positiveFormatString = format.replaceAll("-", "")
+      s"$positiveFormatString;$format"
+    } else {
+      format
+    }
+  }
+
+  protected def check(normalizedFormat: String, numberFormat: String) = {
+    def invalidSignPosition(format: String, c: Char): Boolean = {
+      val signIndex = format.indexOf(c)
+      signIndex > 0 && signIndex < format.length - 1
+    }
+
+    if (normalizedFormat.length == 0) {
+      throw QueryCompilationErrors.emptyNumberFormatError()

Review comment:
       shall we return `TypeCheckResult.TypeCheckFailure` here? These exceptions are not user-facing as we try-catch them immediately.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784470211



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,182 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))

Review comment:
       We have `checkExceptionInExpression` to test errors in both interpreted and codegen.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784468567



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.sql.AnalysisException
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatter(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private lazy val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {
+    val decimalFormat = new DecimalFormat(transformedFormat)
+    decimalFormat.setParseBigDecimal(true)
+    decimalFormat
+  }
+
+  private lazy val precision = normalizedNumberFormat.filterNot(isSign).length
+
+  private lazy val scale = {
+    val formatSplits = normalizedNumberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
+    }
+  }
+
+  def parsedDecimalType: DecimalType = DecimalType(precision, scale)
+
+  /**
+   * DecimalFormat provides '#' and '0' as placeholder of digit, ',' as grouping separator,
+   * '.' as decimal separator, '-' as minus, '$' as dollar, but not '9', 'G', 'D', 'S'. So we need
+   * replace them show below:
+   * 1. '9' -> '#'
+   * 2. 'G' -> ','
+   * 3. 'D' -> '.'
+   * 4. 'S' -> '-'
+   *
+   * Note: When calling format, we must preserve the digits after decimal point, so the digits
+   * after decimal point should be replaced as '0'. For example: '999.9' will be normalized as
+   * '###.0' and '999.99' will be normalized as '###.00', so if the input is 454, the format
+   * output will be 454.0 and 454.00 respectively.
+   *
+   * @param format number format string
+   * @return normalized number format string
+   */
+  private def normalize(format: String): String = {
+    var notFindDecimalPoint = true
+    val normalizedFormat = format.toUpperCase(Locale.ROOT).map {
+      case NINE_DIGIT if notFindDecimalPoint => POUND_SIGN
+      case NINE_DIGIT if !notFindDecimalPoint => ZERO_DIGIT
+      case COMMA_LETTER => COMMA_SIGN
+      case POINT_LETTER | POINT_SIGN =>
+        notFindDecimalPoint = false
+        POINT_SIGN
+      case MINUS_LETTER => MINUS_SIGN
+      case other => other
+    }
+    // If the comma is at the beginning or end of number format, then DecimalFormat will be
+    // invalid. For example, "##,###," or ",###,###" for DecimalFormat is invalid, so we must use
+    // "##,###" or "###,###".
+    normalizedFormat.stripPrefix(COMMA_SIGN_STRING).stripSuffix(COMMA_SIGN_STRING)
+  }
+
+  private def isSign(c: Char): Boolean = {
+    SIGN_SET.contains(c)
+  }
+
+  private def transform(format: String): String = {
+    if (format.contains(MINUS_SIGN)) {
+      // For example: '#.######' represents a positive number,
+      // but '#.######;#.######-' represents a negative number.
+      val positiveFormatString = format.replaceAll("-", "")
+      s"$positiveFormatString;$format"
+    } else {
+      format
+    }
+  }
+
+  def check(): TypeCheckResult = {
+    def invalidSignPosition(c: Char): Boolean = {
+      val signIndex = normalizedNumberFormat.indexOf(c)
+      signIndex > 0 && signIndex < normalizedNumberFormat.length - 1
+    }
+
+    def multipleSignInNumberFormatError(message: String): String = {
+      s"Multiple $message in '$originNumberFormat'"
+    }
+
+    def nonFistOrLastCharInNumberFormatError(message: String): String = {
+      s"$message must be the first or last char in '$originNumberFormat'"
+    }
+
+    if (normalizedNumberFormat.length == 0) {
+      TypeCheckResult.TypeCheckFailure("Number format cannot be empty")
+    } else if (normalizedNumberFormat.count(_ == POINT_SIGN) > 1) {
+      TypeCheckResult.TypeCheckFailure(
+        multipleSignInNumberFormatError(s"'$POINT_LETTER' or '$POINT_SIGN'"))
+    } else if (normalizedNumberFormat.count(_ == MINUS_SIGN) > 1) {
+      TypeCheckResult.TypeCheckFailure(
+        multipleSignInNumberFormatError(s"'$MINUS_LETTER' or '$MINUS_SIGN'"))
+    } else if (normalizedNumberFormat.count(_ == DOLLAR_SIGN) > 1) {
+      TypeCheckResult.TypeCheckFailure(multipleSignInNumberFormatError(s"'$DOLLAR_SIGN'"))
+    } else if (invalidSignPosition(MINUS_SIGN)) {
+      TypeCheckResult.TypeCheckFailure(
+        nonFistOrLastCharInNumberFormatError(s"'$MINUS_LETTER' or '$MINUS_SIGN'"))
+    } else if (invalidSignPosition(DOLLAR_SIGN)) {
+      TypeCheckResult.TypeCheckFailure(
+        nonFistOrLastCharInNumberFormatError(s"'$DOLLAR_SIGN'"))
+    } else {
+      TypeCheckResult.TypeCheckSuccess
+    }
+  }
+
+  /**
+   * Convert string to numeric based on the given number format.
+   * The format can consist of the following characters:
+   * '0' or '9': digit position
+   * '.' or 'D': decimal point (only allowed once)
+   * ',' or 'G': group (thousands) separator
+   * '-' or 'S': sign anchored to number (only allowed once)
+   * '$': value with a leading dollar sign (only allowed once)
+   *
+   * @param input the string need to converted
+   * @return decimal obtained from string parsing
+   */
+  def parse(input: UTF8String): Decimal = {
+    val inputStr = input.toString.trim
+    val inputSplits = inputStr.split(POINT_SIGN)

Review comment:
       let's add `assert(inputSplits.length <= 2)`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784467082



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.sql.AnalysisException
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatter(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private lazy val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {

Review comment:
       I think this is the only one that needs to be `lazy val`. Otherwise are pretty cheap the calculate and can be `val` directly




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786120191



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,179 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+
+    Seq("454", "054", "54", "450").foreach { input =>
+      val invalidFormat1 = 0.until(input.length - 1).map(_ => '0').mkString
+      val invalidFormat2 = 0.until(input.length - 2).map(_ => '0').mkString
+      val invalidFormat3 = 0.until(input.length - 1).map(_ => '9').mkString
+      val invalidFormat4 = 0.until(input.length - 2).map(_ => '9').mkString
+      Seq(invalidFormat1, invalidFormat2, invalidFormat3, invalidFormat4)
+        .filter(_.nonEmpty).foreach { format =>
+        checkExceptionInExpression[IllegalArgumentException](
+          ToNumber(Literal("454"), Literal(format)), s"Format '$format' used for" +
+            " parsing string to number or formatting number to string is invalid")
+      }
+
+      val format1 = 0.until(input.length).map(_ => '0').mkString
+      val format2 = 0.until(input.length).map(_ => '9').mkString
+      val format3 = 0.until(input.length + 1).map(_ => '0').mkString
+      val format4 = 0.until(input.length + 1).map(_ => '9').mkString
+      val format5 = 0.until(input.length + 2).map(_ => '0').mkString
+      val format6 = 0.until(input.length + 2).map(_ => '9').mkString
+      Seq(format1, format2, format3, format4, format5, format6).foreach { format =>
+        checkEvaluation(ToNumber(Literal(input), Literal(format)), Decimal(input))
+      }
+    }
+
+    // Test '.' and 'D'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.2"), Literal("999")),
+      "Format '999' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("999.9")),
+      "Format '999.9' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("000.0")),
+      "Format '000.0' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("000D0")),
+      "Format '000D0' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("00.00")),
+      "Format '00.00' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("00D00")),
+      "Format '00D00' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("0000.0")),
+      "Format '0000.0' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("0000D0")),
+      "Format '0000D0' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("00.000")),
+      "Format '00.000' used for parsing string to number or formatting number to string is invalid")
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("454.23"), Literal("00D000")),
+      "Format '00D000' used for parsing string to number or formatting number to string is invalid")
+
+    Seq(
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),
+      ("454.23", "000.00") -> Decimal(454.23),
+      ("454.23", "000D00") -> Decimal(454.23),
+      ("454.2", "000.00") -> Decimal(454.2),
+      ("454.2", "000D00") -> Decimal(454.2),
+      ("454.0", "000.0") -> Decimal(454),
+      ("454.0", "000D0") -> Decimal(454),
+      ("454.00", "000.00") -> Decimal(454),
+      ("454.00", "000D00") -> Decimal(454),
+      (".4542", ".0000") -> Decimal(0.4542),
+      (".4542", "D0000") -> Decimal(0.4542),
+      ("4542.", "0000.") -> Decimal(4542),
+      ("4542.", "0000D") -> Decimal(4542)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    Seq("999.9.9", "999D9D9", "999.9D9", "999D9.9").foreach { str =>
+      ToNumber(Literal("454.3.2"), Literal(str)).checkInputDataTypes() match {
+        case TypeCheckResult.TypeCheckFailure(msg) =>
+          assert(msg.contains(s"At most one 'D' or '.' is allowed in the number format: '$str'"))
+      }
+    }
+
+    // Test ',' and 'G'
+    checkExceptionInExpression[IllegalArgumentException](
+      ToNumber(Literal("123,456"), Literal("9G9")),
+      "Format '9G9' used for parsing string to number or formatting number to string is invalid")
+
+    Seq(
+      ("12,454", "99,999") -> Decimal(12454),
+      ("12,454", "00,000") -> Decimal(12454),
+      ("12,454", "99G999") -> Decimal(12454),
+      ("12,454", "00G000") -> Decimal(12454),

Review comment:
       same problem here, please make the tests more readable.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786108113



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatter(originNumberFormat: String) extends Serializable {
+  import NumberConstants._
+
+  protected val normalizedNumberFormat = normalize(originNumberFormat)
+
+  private val transformedFormat = transform(normalizedNumberFormat)
+
+  private lazy val numberDecimalFormat = {
+    val decimalFormat = new DecimalFormat(transformedFormat)
+    decimalFormat.setParseBigDecimal(true)
+    decimalFormat
+  }
+
+  private val precision = normalizedNumberFormat.filterNot(isSign).length
+
+  private val scale = {
+    val formatSplits = normalizedNumberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
+    }
+  }
+
+  def parsedDecimalType: DecimalType = DecimalType(precision, scale)

Review comment:
       What I expect is
   ```
   val parsedDecimalType = {
     val formatSplits = normalizedNumberFormat.split(POINT_SIGN).map(_.filterNot(isSign))
     assert(formatSplits.length <= 2)
     val precision = formatSplits.map(_.length).sum
     val scale = if (formatSplits.length == 2) formatSplits.last.length else 0
     DecimalType(precision, scale)
   }
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786595587



##########
File path: sql/core/src/test/resources/sql-tests/inputs/string-functions.sql
##########
@@ -124,4 +124,76 @@ SELECT endswith('Spark SQL', 'QL');
 SELECT endswith('Spark SQL', 'Spa');
 SELECT endswith(null, 'Spark');
 SELECT endswith('Spark', null);
-SELECT endswith(null, null);
\ No newline at end of file
+SELECT endswith(null, null);
+
+-- to_number

Review comment:
       OK




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786478459



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.sql.AnalysisException
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {

Review comment:
       Updated




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35060:
URL: https://github.com/apache/spark/pull/35060#issuecomment-1017272200


   thanks, merging to master!


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776718714



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)

Review comment:
       `to_number` is a parsing function, I think 0 and 9 have no difference?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776930869



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)

Review comment:
       > Then why do we document them differently here?
   
   Updated




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781169711



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '0' or '9': digit position
+         '.' or 'D':  decimal point (only allowed once)
+         ',' or 'G':  group (thousands) separator
+         '-' or 'S':  sign anchored to number
+         '$': returns value with a leading dollar sign
+  """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_('454', '999');
+       454
+      > SELECT _FUNC_('454.00', '000D00');
+       454.00
+      > SELECT _FUNC_('12,454', '99G999');
+       12454
+      > SELECT _FUNC_('$78.12', '$99.99');
+       78.12
+      > SELECT _FUNC_('12,454.8-', '99G999D9S');
+       -12454.8
+  """,
+  since = "3.3.0",
+  group = "string_funcs")
+case class ToNumber(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant {
+
+  private lazy val numberFormat = right.eval().toString.toUpperCase(Locale.ROOT)
+  private lazy val numberFormatBuilder = new NumberFormatBuilder(numberFormat)
+  private lazy val (precision, scale) = numberFormatBuilder.parsePrecisionAndScale()
+
+  override def dataType: DataType = DecimalType(precision, scale)
+
+  override def inputTypes: Seq[DataType] = Seq(StringType, StringType)
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val inputTypeCheck = super.checkInputDataTypes()
+    if (inputTypeCheck.isSuccess) {
+      if (right.foldable) {
+        numberFormatBuilder.check()
+      } else {
+        TypeCheckResult.TypeCheckFailure(s"Format expression must be foldable, but got $right")
+      }
+    } else {
+      inputTypeCheck
+    }
+  }
+
+  override def prettyName: String = "to_number"
+
+  override def nullSafeEval(string: Any, format: Any): Any = {
+    val input = string.asInstanceOf[UTF8String]
+    numberFormatBuilder.parse(input)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+    val builder =
+      ctx.addReferenceObj("builder", numberFormatBuilder, classOf[NumberFormatBuilder].getName)
+    val eval = left.genCode(ctx)
+    ev.copy(code = code"""

Review comment:
       nit: use multi-line string.
   ```
   code"""
     |...
     """.stripMargin
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r781171223



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -21,49 +21,73 @@ import java.math.BigDecimal
 import java.text.{DecimalFormat, NumberFormat, ParsePosition}
 import java.util.Locale
 
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
 import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
 import org.apache.spark.sql.types.Decimal
 import org.apache.spark.unsafe.types.UTF8String
 
 object NumberUtils {
 
-  private val pointSign = '.'
-  private val letterPointSign = 'D'
-  private val commaSign = ','
-  private val letterCommaSign = 'G'
-  private val minusSign = '-'
-  private val letterMinusSign = 'S'
-  private val dollarSign = '$'
+  private final val POINT_SIGN = '.'
+  private final val POINT_LETTER = 'D'
+  private final val COMMA_SIGN = ','
+  private final val COMMA_LETTER = 'G'
+  private final val MINUS_SIGN = '-'
+  private final val MINUS_LETTER = 'S'
+  private final val DOLLAR_SIGN = '$'
+  private final val NINE_DIGIT = '9'
+  private final val ZERO_DIGIT = '0'
+  private final val POUND_SIGN = '#'
 
-  private val commaSignStr = commaSign.toString
+  private final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  private final val POUND_SIGN_STRING = POUND_SIGN.toString
 
+  private final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+
+  /**
+   * DecimalFormat provides '#' and '0' as placeholder of digit, ',' as grouping separator,
+   * '.' as decimal separator, '-' as minus, '$' as dollar, but '9', 'G', 'D', 'S'. So we need

Review comment:
       ```suggestion
      * '.' as decimal separator, '-' as minus, '$' as dollar, but not '9', 'G', 'D', 'S'. So we need
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r782846377



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -77,80 +107,91 @@ object NumberUtils {
       signIndex > 0 && signIndex < format.length - 1
     }
 
-    if (normalizedFormat.count(_ == pointSign) > 1) {
+    if (normalizedFormat.length == 0) {
+      throw QueryCompilationErrors.emptyNumberFormatError()
+    } else if (normalizedFormat.count(_ == POINT_SIGN) > 1) {
       throw QueryCompilationErrors.multipleSignInNumberFormatError(
-        s"'$letterPointSign' or '$pointSign'", numberFormat)
-    } else if (normalizedFormat.count(_ == minusSign) > 1) {
+        s"'$POINT_LETTER' or '$POINT_SIGN'", numberFormat)
+    } else if (normalizedFormat.count(_ == MINUS_SIGN) > 1) {
       throw QueryCompilationErrors.multipleSignInNumberFormatError(
-        s"'$letterMinusSign' or '$minusSign'", numberFormat)
-    } else if (normalizedFormat.count(_ == dollarSign) > 1) {
-      throw QueryCompilationErrors.multipleSignInNumberFormatError(s"'$dollarSign'", numberFormat)
-    } else if (invalidSignPosition(normalizedFormat, minusSign)) {
+        s"'$MINUS_LETTER' or '$MINUS_SIGN'", numberFormat)
+    } else if (normalizedFormat.count(_ == DOLLAR_SIGN) > 1) {
+      throw QueryCompilationErrors.multipleSignInNumberFormatError(s"'$DOLLAR_SIGN'", numberFormat)
+    } else if (invalidSignPosition(normalizedFormat, MINUS_SIGN)) {
       throw QueryCompilationErrors.nonFistOrLastCharInNumberFormatError(
-        s"'$letterMinusSign' or '$minusSign'", numberFormat)
-    } else if (invalidSignPosition(normalizedFormat, dollarSign)) {
+        s"'$MINUS_LETTER' or '$MINUS_SIGN'", numberFormat)
+    } else if (invalidSignPosition(normalizedFormat, DOLLAR_SIGN)) {
       throw QueryCompilationErrors.nonFistOrLastCharInNumberFormatError(
-        s"'$dollarSign'", numberFormat)
+        s"'$DOLLAR_SIGN'", numberFormat)
+    }
+  }
+
+  private def getPrecision(numberFormat: String): Int =
+    numberFormat.filterNot(isSign).length
+
+  private def getScale(numberFormat: String): Int = {
+    val formatSplits = numberFormat.split(POINT_SIGN)
+    if (formatSplits.length == 1) {
+      0
+    } else {
+      formatSplits(1).filterNot(isSign).length
     }
   }
 
   /**
    * Convert string to numeric based on the given number format.
    * The format can consist of the following characters:
-   * '9':  digit position (can be dropped if insignificant)
-   * '0':  digit position (will not be dropped, even if insignificant)
-   * '.':  decimal point (only allowed once)
-   * ',':  group (thousands) separator
-   * 'S':  sign anchored to number (uses locale)
-   * 'D':  decimal point (uses locale)
-   * 'G':  group separator (uses locale)
-   * '$':  specifies that the input value has a leading $ (Dollar) sign.
+   * '0' or '9': digit position
+   * '.' or 'D': decimal point (only allowed once)
+   * ',' or 'G': group (thousands) separator
+   * '-' or 'S': sign anchored to number (only allowed once)
+   * '$': value with a leading dollar sign (only allowed once)
    *
    * @param input the string need to converted
-   * @param numberFormat the given number format
+   * @param originNumberFormat the origin number format
+   * @param normalizedNumberFormat normalized number format
+   * @param precision decimal precision
+   * @param scale decimal scale
    * @return decimal obtained from string parsing
    */
-  def parse(input: UTF8String, numberFormat: String): Decimal = {
-    val normalizedFormat = normalize(numberFormat)
-    check(normalizedFormat, numberFormat)
-
-    val precision = normalizedFormat.filterNot(isSign).length
-    val formatSplits = normalizedFormat.split(pointSign)
-    val scale = if (formatSplits.length == 1) {
-      0
-    } else {
-      formatSplits(1).filterNot(isSign).length
-    }
-    val transformedFormat = transform(normalizedFormat)
-    val numberFormatInstance = NumberFormat.getInstance()
-    val numberDecimalFormat = numberFormatInstance.asInstanceOf[DecimalFormat]
-    numberDecimalFormat.setParseBigDecimal(true)
-    numberDecimalFormat.applyPattern(transformedFormat)
+  private def parse(
+      input: UTF8String,
+      originNumberFormat: String,
+      normalizedNumberFormat: String,
+      precision: Int,
+      scale: Int): Decimal = {
     val inputStr = input.toString.trim
-    val inputSplits = inputStr.split(pointSign)
+    val inputSplits = inputStr.split(POINT_SIGN)
     if (inputSplits.length == 1) {
       if (inputStr.filterNot(isSign).length > precision - scale) {
-        throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
+        throw QueryExecutionErrors.invalidNumberFormatError(originNumberFormat)
       }
     } else if (inputSplits(0).filterNot(isSign).length > precision - scale ||
       inputSplits(1).filterNot(isSign).length > scale) {
-      throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
+      throw QueryExecutionErrors.invalidNumberFormatError(originNumberFormat)
+    }
+
+    val transformedFormat = transform(normalizedNumberFormat)
+    try {
+      numberDecimalFormat.applyLocalizedPattern(transformedFormat)

Review comment:
       Updated




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784472530



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,182 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("99")))
+    }
+
+    Seq(
+      ("454", "999") -> Decimal(454),
+      ("054", "999") -> Decimal(54),
+      ("54", "999") -> Decimal(54),
+      ("404", "999") -> Decimal(404),
+      ("450", "999") -> Decimal(450),
+      ("454", "9999") -> Decimal(454),
+      ("054", "9999") -> Decimal(54),
+      ("404", "9999") -> Decimal(404),
+      ("450", "9999") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("0")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("00")))
+    }
+
+    Seq(

Review comment:
       I'd write the test in this way
   ```
   Seq("454", "054", "54", "450").foreach { input =>
     val format1 = input.map(_ => '0')
     val format2 = input.map(_ => '9')
     val format3 = 0.until(input.length + 2).map(i => i % 2).mkString
     Seq(format1, format2, format3).foreach { format => 
       checkEvaluation(ToNumber(Literal(input), Literal(format)), Decimal(input))
     }
   }
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784565837



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,182 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("99")))
+    }
+
+    Seq(
+      ("454", "999") -> Decimal(454),
+      ("054", "999") -> Decimal(54),
+      ("54", "999") -> Decimal(54),
+      ("404", "999") -> Decimal(404),
+      ("450", "999") -> Decimal(450),
+      ("454", "9999") -> Decimal(454),
+      ("054", "9999") -> Decimal(54),
+      ("404", "9999") -> Decimal(404),
+      ("450", "9999") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("0")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("00")))
+    }
+
+    Seq(
+      ("454", "000") -> Decimal(454),
+      ("054", "000") -> Decimal(54),
+      ("54", "000") -> Decimal(54),
+      ("404", "000") -> Decimal(404),
+      ("450", "000") -> Decimal(450),
+      ("454", "0000") -> Decimal(454),
+      ("054", "0000") -> Decimal(54),
+      ("404", "0000") -> Decimal(404),
+      ("450", "0000") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    // Test '.' and 'D'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.2"), Literal("999")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.23"), Literal("999.9")))
+    }
+
+    Seq(
+      ("454.2", "999.9") -> Decimal(454.2),
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "999D9") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),

Review comment:
       OK




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783693188



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -164,26 +192,74 @@ object NumberUtils {
     val bigDecimal = input.toJavaBigDecimal
     val decimalPlainStr = bigDecimal.toPlainString
     if (decimalPlainStr.length > transformedFormat.length) {
-      transformedFormat.replaceAll("0", "#")
+      transformedFormat.replaceAll("0", POUND_SIGN_STRING)
     } else {
-      val decimalFormat = new DecimalFormat(transformedFormat)
-      var resultStr = decimalFormat.format(bigDecimal)
+      val numberDecimalFormat = {
+        val decimalFormat = new DecimalFormat()
+        decimalFormat.setParseBigDecimal(true)
+        try {
+          decimalFormat.applyLocalizedPattern(transformedFormat)
+        } catch {
+          case _: IllegalArgumentException =>
+            throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
+        }
+        decimalFormat
+      }
+      var resultStr = numberDecimalFormat.format(bigDecimal)
       // Since we trimmed the comma at the beginning or end of number format in function
       // `normalize`, we restore the comma to the result here.
       // For example, if the specified number format is "99,999," or ",999,999", function
       // `normalize` normalize them to "##,###" or "###,###".
       // new DecimalFormat("##,###").parse(12454) and new DecimalFormat("###,###").parse(124546)
       // will return "12,454" and "124,546" respectively. So we add ',' at the end and head of
       // the result, then the final output are "12,454," or ",124,546".
-      if (numberFormat.last == commaSign || numberFormat.last == letterCommaSign) {
-        resultStr = resultStr + commaSign
+      if (numberFormat.last == COMMA_SIGN || numberFormat.last == COMMA_LETTER) {
+        resultStr = resultStr + COMMA_SIGN
       }
-      if (numberFormat.charAt(0) == commaSign || numberFormat.charAt(0) == letterCommaSign) {
-        resultStr = commaSign + resultStr
+      if (numberFormat.charAt(0) == COMMA_SIGN || numberFormat.charAt(0) == COMMA_LETTER) {
+        resultStr = COMMA_SIGN + resultStr
       }
 
       resultStr
     }
   }
 
+  class NumberFormatBuilder(originNumberFormat: String) extends Serializable {

Review comment:
       do we need `NumberUtils` at all?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784423563



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
##########
@@ -888,6 +889,174 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null)
   }
 
+  test("ToNumber") {
+    ToNumber(Literal("454"), Literal("")).checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Number format cannot be empty"))
+    }
+    ToNumber(Literal("454"), NonFoldableLiteral.create("999", StringType))
+      .checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("Format expression must be foldable"))
+    }
+
+    // Test '0' and '9'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("9")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("99")))
+    }
+
+    Seq(
+      ("454", "999") -> Decimal(454),
+      ("054", "999") -> Decimal(54),
+      ("54", "999") -> Decimal(54),
+      ("404", "999") -> Decimal(404),
+      ("450", "999") -> Decimal(450),
+      ("454", "9999") -> Decimal(454),
+      ("054", "9999") -> Decimal(54),
+      ("404", "9999") -> Decimal(404),
+      ("450", "9999") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("0")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454"), Literal("00")))
+    }
+
+    Seq(
+      ("454", "000") -> Decimal(454),
+      ("054", "000") -> Decimal(54),
+      ("54", "000") -> Decimal(54),
+      ("404", "000") -> Decimal(404),
+      ("450", "000") -> Decimal(450),
+      ("454", "0000") -> Decimal(454),
+      ("054", "0000") -> Decimal(54),
+      ("404", "0000") -> Decimal(404),
+      ("450", "0000") -> Decimal(450)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    // Test '.' and 'D'
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.2"), Literal("999")))
+    }
+    intercept[IllegalArgumentException] {
+      evaluateWithoutCodegen(ToNumber(Literal("454.23"), Literal("999.9")))
+    }
+
+    Seq(
+      ("454.2", "999.9") -> Decimal(454.2),
+      ("454.2", "000.0") -> Decimal(454.2),
+      ("454.2", "999D9") -> Decimal(454.2),
+      ("454.2", "000D0") -> Decimal(454.2),
+      ("454.23", "999.99") -> Decimal(454.23),
+      ("454.23", "000.00") -> Decimal(454.23),
+      ("454.23", "999D99") -> Decimal(454.23),
+      ("454.23", "000D00") -> Decimal(454.23),
+      ("454.0", "999.9") -> Decimal(454),
+      ("454.0", "000.0") -> Decimal(454),
+      ("454.0", "999D9") -> Decimal(454),
+      ("454.0", "000D0") -> Decimal(454),
+      ("454.00", "999.99") -> Decimal(454),
+      ("454.00", "000.00") -> Decimal(454),
+      ("454.00", "999D99") -> Decimal(454),
+      ("454.00", "000D00") -> Decimal(454),
+      (".4542", ".9999") -> Decimal(0.4542),
+      (".4542", ".0000") -> Decimal(0.4542),
+      (".4542", "D9999") -> Decimal(0.4542),
+      (".4542", "D0000") -> Decimal(0.4542),
+      ("4542.", "9999.") -> Decimal(4542),
+      ("4542.", "0000.") -> Decimal(4542),
+      ("4542.", "9999D") -> Decimal(4542),
+      ("4542.", "0000D") -> Decimal(4542)
+    ).foreach { case ((str, format), expected) =>
+      checkEvaluation(ToNumber(Literal(str), Literal(format)), expected)
+    }
+
+    Seq("999.9.9", "999D9D9", "999.9D9", "999D9.9").foreach { str =>
+      ToNumber(Literal("454.3.2"), Literal(str)).checkInputDataTypes() match {
+        case TypeCheckResult.TypeCheckFailure(msg) =>
+          assert(msg.contains(s"Multiple 'D' or '.' in '$str'"))
+      }
+    }
+
+    // Test ',' and 'G'
+    Seq(
+      ("12,454", "99,999") -> Decimal(12454),
+      ("12,454", "00,000") -> Decimal(12454),
+      ("12,454", "99G999") -> Decimal(12454),
+      ("12,454", "00G000") -> Decimal(12454),
+      ("12,454,367", "99,999,999") -> Decimal(12454367),
+      ("12,454,367", "00,000,000") -> Decimal(12454367),
+      ("12,454,367", "99G999G999") -> Decimal(12454367),
+      ("12,454,367", "00G000G000") -> Decimal(12454367),
+      ("12,454,", "99,999,") -> Decimal(12454),
+      ("12,454,", "00,000,") -> Decimal(12454),
+      ("12,454,", "99G999G") -> Decimal(12454),
+      ("12,454,", "00G000G") -> Decimal(12454),
+      (",454,367", ",999,999") -> Decimal(454367),

Review comment:
       OK




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r784003805



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {
+  final val POINT_SIGN = '.'
+  final val POINT_LETTER = 'D'
+  final val COMMA_SIGN = ','
+  final val COMMA_LETTER = 'G'
+  final val MINUS_SIGN = '-'
+  final val MINUS_LETTER = 'S'
+  final val DOLLAR_SIGN = '$'
+  final val NINE_DIGIT = '9'
+  final val ZERO_DIGIT = '0'
+  final val POUND_SIGN = '#'
+
+  final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  final val POUND_SIGN_STRING = POUND_SIGN.toString
+
+  final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+}
+
+class NumberFormatBuilder(originNumberFormat: String) extends Serializable {

Review comment:
       the name builder is a bit weird, `NumberFormatter` sounds better.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783639634



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -18,52 +18,76 @@
 package org.apache.spark.sql.catalyst.util
 
 import java.math.BigDecimal
-import java.text.{DecimalFormat, NumberFormat, ParsePosition}
+import java.text.{DecimalFormat, ParsePosition}
 import java.util.Locale
 
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
 import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
 import org.apache.spark.sql.types.Decimal
 import org.apache.spark.unsafe.types.UTF8String
 
 object NumberUtils {
 
-  private val pointSign = '.'
-  private val letterPointSign = 'D'
-  private val commaSign = ','
-  private val letterCommaSign = 'G'
-  private val minusSign = '-'
-  private val letterMinusSign = 'S'
-  private val dollarSign = '$'
+  private final val POINT_SIGN = '.'
+  private final val POINT_LETTER = 'D'
+  private final val COMMA_SIGN = ','
+  private final val COMMA_LETTER = 'G'
+  private final val MINUS_SIGN = '-'
+  private final val MINUS_LETTER = 'S'
+  private final val DOLLAR_SIGN = '$'
+  private final val NINE_DIGIT = '9'
+  private final val ZERO_DIGIT = '0'
+  private final val POUND_SIGN = '#'
 
-  private val commaSignStr = commaSign.toString
+  private final val COMMA_SIGN_STRING = COMMA_SIGN.toString
+  private final val POUND_SIGN_STRING = POUND_SIGN.toString
 
+  private final val SIGN_SET = Set(POINT_SIGN, COMMA_SIGN, MINUS_SIGN, DOLLAR_SIGN)
+
+  /**
+   * DecimalFormat provides '#' and '0' as placeholder of digit, ',' as grouping separator,
+   * '.' as decimal separator, '-' as minus, '$' as dollar, but not '9', 'G', 'D', 'S'. So we need
+   * replace them show below:
+   * 1. '9' -> '#'
+   * 2. 'G' -> ','
+   * 3. 'D' -> '.'
+   * 4. 'S' -> '-'
+   *
+   * Note: When calling format, we must preserve the digits after decimal point, so the digits
+   * after decimal point should be replaced as '0'. For example: '999.9' will be normalized as
+   * '###.0' and '999.99' will be normalized as '###.00', so if the input is 454, the format
+   * output will be 454.0 and 454.00 respectively.
+   *
+   * @param format number format string
+   * @return normalized number format string
+   */
   private def normalize(format: String): String = {
-    var notFindDecimalPoint = true
+    var flag = true

Review comment:
       this name is too vague. the previous name is better.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r783659930



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberUtils.scala
##########
@@ -164,26 +192,74 @@ object NumberUtils {
     val bigDecimal = input.toJavaBigDecimal
     val decimalPlainStr = bigDecimal.toPlainString
     if (decimalPlainStr.length > transformedFormat.length) {
-      transformedFormat.replaceAll("0", "#")
+      transformedFormat.replaceAll("0", POUND_SIGN_STRING)
     } else {
-      val decimalFormat = new DecimalFormat(transformedFormat)
-      var resultStr = decimalFormat.format(bigDecimal)
+      val numberDecimalFormat = {
+        val decimalFormat = new DecimalFormat()
+        decimalFormat.setParseBigDecimal(true)
+        try {
+          decimalFormat.applyLocalizedPattern(transformedFormat)
+        } catch {
+          case _: IllegalArgumentException =>
+            throw QueryExecutionErrors.invalidNumberFormatError(numberFormat)
+        }
+        decimalFormat
+      }
+      var resultStr = numberDecimalFormat.format(bigDecimal)
       // Since we trimmed the comma at the beginning or end of number format in function
       // `normalize`, we restore the comma to the result here.
       // For example, if the specified number format is "99,999," or ",999,999", function
       // `normalize` normalize them to "##,###" or "###,###".
       // new DecimalFormat("##,###").parse(12454) and new DecimalFormat("###,###").parse(124546)
       // will return "12,454" and "124,546" respectively. So we add ',' at the end and head of
       // the result, then the final output are "12,454," or ",124,546".
-      if (numberFormat.last == commaSign || numberFormat.last == letterCommaSign) {
-        resultStr = resultStr + commaSign
+      if (numberFormat.last == COMMA_SIGN || numberFormat.last == COMMA_LETTER) {
+        resultStr = resultStr + COMMA_SIGN
       }
-      if (numberFormat.charAt(0) == commaSign || numberFormat.charAt(0) == letterCommaSign) {
-        resultStr = commaSign + resultStr
+      if (numberFormat.charAt(0) == COMMA_SIGN || numberFormat.charAt(0) == COMMA_LETTER) {
+        resultStr = COMMA_SIGN + resultStr
       }
 
       resultStr
     }
   }
 
+  class NumberFormatBuilder(originNumberFormat: String) extends Serializable {

Review comment:
       `NumberFormatBuilder` used to keep context of number format and some cache.
   `NumberFormatBuilder` also used as the public API for `parse` and `format`.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r776719568



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.util.NumberUtils.NumberFormatBuilder
+import org.apache.spark.sql.types.{DataType, DecimalType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A function that converts string to numeric.
+ */
+@ExpressionDescription(
+  usage = """
+     _FUNC_(strExpr, formatExpr) - Convert `strExpr` to a number based on the `formatExpr`.
+       The format can consist of the following characters:
+         '9':  digit position (can be dropped if insignificant)
+         '0':  digit position (will not be dropped, even if insignificant)
+         '.':  decimal point (only allowed once)
+         ',':  group (thousands) separator
+         'S':  sign anchored to number (uses locale)
+         'D':  decimal point (uses locale)

Review comment:
       I only see that we replace `D` with `.` during normalization.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r788313185



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberFormatterSuite.scala
##########
@@ -19,76 +19,75 @@ package org.apache.spark.sql.catalyst.util
 
 import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.util.NumberUtils.{format, parse}
 import org.apache.spark.sql.types.Decimal
 import org.apache.spark.unsafe.types.UTF8String
 
-class NumberUtilsSuite extends SparkFunSuite {
+class NumberFormatterSuite extends SparkFunSuite {

Review comment:
       Just follows `StringUtils` with `StringUtilsSuite`.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r788582373



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberFormatterSuite.scala
##########
@@ -19,76 +19,75 @@ package org.apache.spark.sql.catalyst.util
 
 import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.util.NumberUtils.{format, parse}
 import org.apache.spark.sql.types.Decimal
 import org.apache.spark.unsafe.types.UTF8String
 
-class NumberUtilsSuite extends SparkFunSuite {
+class NumberFormatterSuite extends SparkFunSuite {
 
-  private def failParseWithInvalidInput(
-      input: UTF8String, numberFormat: String, errorMsg: String): Unit = {
-    val e = intercept[IllegalArgumentException](parse(input, numberFormat))
+  private def invalidNumberFormat(numberFormat: String, errorMsg: String): Unit = {
+    val testNumberFormatter = new TestNumberFormatter(numberFormat)

Review comment:
       OK




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786103581



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConstants.scala
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.spark.sql.catalyst.util
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.sql.AnalysisException
+
+import java.math.BigDecimal
+import java.text.{DecimalFormat, ParsePosition}
+import java.util.Locale
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types.{Decimal, DecimalType}
+import org.apache.spark.unsafe.types.UTF8String
+
+object NumberConstants {

Review comment:
       hmm, I can't find it




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r786121415



##########
File path: sql/core/src/test/resources/sql-tests/inputs/string-functions.sql
##########
@@ -124,4 +124,76 @@ SELECT endswith('Spark SQL', 'QL');
 SELECT endswith('Spark SQL', 'Spa');
 SELECT endswith(null, 'Spark');
 SELECT endswith('Spark', null);
-SELECT endswith(null, null);
\ No newline at end of file
+SELECT endswith(null, null);
+
+-- to_number

Review comment:
       since we have extensive unit tests, we can only test some common cases in the end-to-end tests




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779598055



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|
+|**0**|digit position (will not be dropped, even if insignificant)|0999|
+|**.**|decimal point (only allowed once)|99.99|
+|**D**|decimal point (only allowed once)|99D99|

Review comment:
       ```suggestion
   |**D**|decimal point, same as **.** (only allowed once)|99D99|
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35060:
URL: https://github.com/apache/spark/pull/35060#discussion_r779592689



##########
File path: docs/sql-ref-number-pattern.md
##########
@@ -0,0 +1,40 @@
+---
+layout: global
+title: Number patterns
+displayTitle: Number Patterns for Formatting and Parsing
+license: |
+  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.
+---
+
+Spark uses pattern letters in the following table for number parsing and formatting:
+
+|Symbol|Meaning|Examples|
+|------|-------|--------|
+|**9**|digit position (can be dropped if insignificant)|9999|

Review comment:
       ```suggestion
   |**9**|Position for a digit; leading/trailing zeros are trimmed when formatting.|9999|
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] beliefer commented on pull request #35060: [SPARK-28137][SQL] Data Type Formatting Functions: `to_number`

Posted by GitBox <gi...@apache.org>.
beliefer commented on pull request #35060:
URL: https://github.com/apache/spark/pull/35060#issuecomment-1017303239


   @cloud-fan Thank you for your hard work!


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org