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

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new cbfeff7a5e6 [SPARK-28516][SQL] Implement `to_char` and `try_to_char` 
functions to format Decimal values as strings
cbfeff7a5e6 is described below

commit cbfeff7a5e69d3a90c56b80679ab242a0cbe1220
Author: Daniel Tenedorio <daniel.tenedo...@databricks.com>
AuthorDate: Fri May 13 11:36:58 2022 +0800

    [SPARK-28516][SQL] Implement `to_char` and `try_to_char` functions to 
format Decimal values as strings
    
    ### What changes were proposed in this pull request?
    
    This PR implements the `to_char` and `try_to_char` SQL function expressions 
according to new semantics described below. The former is equivalent to the 
latter except that it throws an exception instead of returning NULL for cases 
where the input decimal value does not match the format string.
    
    -----------
    
    # `try_to_char` function
    
    Returns Decimal-typed `expr` cast to STRING using formatting `fmt`, or 
`NULL` if `expr` is not a valid match for the given format.
    
    ## Syntax
    
    ```
    try_to_char(expr, fmt)
    fmt
      { ' [ S ] [ L | $ ]
          [ 0 | 9 | G | , ] [...]
          [ . | D ]
          [ 0 | 9 ] [...]
          [ L | $ ] [ PR | MI | S ] ' }
    ```
    
    ## Arguments
    
    - `expr`: A DECIMAL expression representing a number. `expr` may include 
leading or trailing spaces.
    - `fmt`: An STRING literal, specifying the expected format of `expr`.
    
    ## Returns
    
    A STRING representing the result of the formatting operation.
    
    `fmt` can contain the following elements (case insensitive):
    
    ```
    '0' or '9':
    
    Specifies an expected digit between 0 and 9. A sequence of 0 or 9 in the 
format
    string matches a sequence of digits in the input value, generating a result 
string
    of the same length as the corresponding sequence in the format string. The 
result
    string is left-padded with zeros if the 0/9 sequence comprises more digits 
than the
    matching part of the decimal value, starts with 0, and is before the 
decimal point.
    Otherwise, it is padded with spaces.
    
    '.' or 'D':
    
    Specifies the position of the decimal point (optional, only allowed once).
    
    ',' or 'G':
    
    Specifies the position of the grouping (thousands) separator (,). There 
must be a
    0 or 9 to the left and right of each grouping separator.
    
    '$':
    
    Specifies the location of the $ currency sign. This character may only be 
specified once.
    
    'S' or 'MI':
    
    Specifies the position of a '-' or '+' sign (optional, only allowed once at 
the
    beginning or end of the format string). Note that 'S' prints '+' for 
positive values
    but 'MI' prints a space.
    
    'PR':
    
    Only allowed at the end of the format string; specifies that the result 
string will
    be wrapped by angle brackets if the input value is negative.  ('<1>').
    ```
    
    If `expr` contains any characters other than `0` through `9` and those 
permitted in `fmt` a `NULL` is returned.
    
    ## Examples
    
    ```sql
    -- The format expects:
    --  * an optional sign at the beginning,
    --  * followed by a dollar sign,
    --  * followed by a number between 3 and 6 digits long,
    --  * thousands separators,
    --  * up to two dights beyond the decimal point.
    Examples:
    > SELECT try_to_char(Decimal(454), '999');
    "454"
    > SELECT try_to_char(Decimal(454.00), '000.00');
    "454.00"
    > SELECT try_to_char(Decimal(12454), '99,999');
    "12,454"
    > SELECT try_to_char(Decimal(78.12), '$99.99');
    "$78.12"
    > SELECT try_to_char(Decimal(-12454.8), '99,999.9S');
    "12,454.8-"
    > SELECT try_to_char(Decimal(12454.8), '99,999.9S');
    NULL
    ```
    
    ### Why are the changes needed?
    
    The new semantics bring Spark into consistency with other engines and grant 
the user flexibility about how to handle cases where inputs do not match the 
format string.
    
    ### Does this PR introduce _any_ user-facing change?
    
    Yes.
    
    * The minus sign `-` is no longer supported in the format string (`S` 
replaces it).
    * `MI` and `PR` are new options in the format string.
    * `to_number` and `try_to_number` are separate functions with different 
error behavior.
    
    ### How was this patch tested?
    
    * New positive and negative unit tests cover both `to_char` and 
`try_to_char` functions.
    * Query tests update as needed according to the behavior changes.
    
    Closes #36365 from dtenedor/to-char.
    
    Authored-by: Daniel Tenedorio <daniel.tenedo...@databricks.com>
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>
---
 .../sql/catalyst/analysis/FunctionRegistry.scala   |   2 +
 .../expressions/numberFormatExpressions.scala      | 162 ++++++++-
 .../spark/sql/catalyst/util/ToNumberParser.scala   | 320 +++++++++++++++--
 .../spark/sql/errors/QueryExecutionErrors.scala    |   4 +-
 .../expressions/StringExpressionsSuite.scala       | 383 ++++++++++++++++++++-
 .../sql-functions/sql-expression-schema.md         |   2 +
 .../resources/sql-tests/inputs/postgreSQL/int8.sql |  53 ++-
 .../sql-tests/inputs/postgreSQL/numeric.sql        |  58 ++--
 .../sql-tests/results/postgreSQL/int8.sql.out      | 182 +++++++++-
 .../sql-tests/results/postgreSQL/numeric.sql.out   | 242 ++++++++++++-
 10 files changed, 1306 insertions(+), 102 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index a56ef175b5e..50f376c0ce6 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -525,6 +525,8 @@ object FunctionRegistry {
     expression[FormatString]("format_string"),
     expression[ToNumber]("to_number"),
     expression[TryToNumber]("try_to_number"),
+    expression[ToCharacter]("to_char"),
+    expression[TryToCharacter]("try_to_char"),
     expression[GetJsonObject]("get_json_object"),
     expression[InitCap]("initcap"),
     expression[StringInstr]("instr"),
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
index c866bb9af9e..0bf573334f2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
@@ -23,7 +23,7 @@ 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.ToNumberParser
-import org.apache.spark.sql.types.{DataType, StringType}
+import org.apache.spark.sql.types.{AbstractDataType, DataType, Decimal, 
DecimalType, StringType}
 import org.apache.spark.unsafe.types.UTF8String
 
 /**
@@ -38,11 +38,11 @@ import org.apache.spark.unsafe.types.UTF8String
          '0' or '9': Specifies an expected digit between 0 and 9. A sequence 
of 0 or 9 in the format
            string matches a sequence of digits in the input string. If the 0/9 
sequence starts with
            0 and is before the decimal point, it can only match a digit 
sequence of the same size.
-           Otherwise, if the sequence starts with 9 or is after the decimal 
poin, it can match a
+           Otherwise, if the sequence starts with 9 or is after the decimal 
point, it can match a
            digit sequence that has the same or smaller size.
          '.' or 'D': Specifies the position of the decimal point (optional, 
only allowed once).
          ',' or 'G': Specifies the position of the grouping (thousands) 
separator (,). There must be
-           one or more 0 or 9 to the left of the rightmost grouping separator. 
'expr' must match the
+           a 0 or 9 to the left and right of each grouping separator. 'expr' 
must match the
            grouping separator relevant for the size of the number.
          '$': Specifies the location of the $ currency sign. This character 
may only be specified
            once.
@@ -168,3 +168,159 @@ case class TryToNumber(left: Expression, right: 
Expression)
       newRight: Expression): TryToNumber =
     copy(left = newLeft, right = newRight)
 }
+
+/**
+ * A function that converts decimal values to strings, returning NULL if the 
decimal value fails to
+ * match the format string.
+ */
+@ExpressionDescription(
+  usage = """
+    _FUNC_(numberExpr, formatExpr) - Convert `numberExpr` to a string based on 
the `formatExpr`.
+      Throws an exception if the conversion fails. The format can consist of 
the following
+      characters, case insensitive:
+        '0' or '9': Specifies an expected digit between 0 and 9. A sequence of 
0 or 9 in the format
+          string matches a sequence of digits in the input value, generating a 
result string of the
+          same length as the corresponding sequence in the format string. The 
result string is
+          left-padded with zeros if the 0/9 sequence comprises more digits 
than the matching part of
+          the decimal value, starts with 0, and is before the decimal point. 
Otherwise, it is
+          padded with spaces.
+        '.' or 'D': Specifies the position of the decimal point (optional, 
only allowed once).
+        ',' or 'G': Specifies the position of the grouping (thousands) 
separator (,). There must be
+          a 0 or 9 to the left and right of each grouping separator.
+        '$': Specifies the location of the $ currency sign. This character may 
only be specified
+          once.
+        'S' or 'MI': Specifies the position of a '-' or '+' sign (optional, 
only allowed once at
+          the beginning or end of the format string). Note that 'S' prints '+' 
for positive values
+          but 'MI' prints a space.
+        'PR': Only allowed at the end of the format string; specifies that the 
result string will be
+          wrapped by angle brackets if the input value is negative.
+          ('<1>').
+  """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_(454, '999');
+       454
+      > SELECT _FUNC_(454.00, '000D00');
+       454.00
+      > SELECT _FUNC_(12454, '99G999');
+       12,454
+      > SELECT _FUNC_(78.12, '$99.99');
+       $78.12
+      > SELECT _FUNC_(-12454.8, '99G999D9S');
+       12,454.8-
+  """,
+  since = "3.4.0",
+  group = "string_funcs")
+case class ToCharacter(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant {
+  private lazy val numberFormat = 
right.eval().toString.toUpperCase(Locale.ROOT)
+  private lazy val numberFormatter = new ToNumberParser(numberFormat, true)
+
+  override def dataType: DataType = StringType
+  override def inputTypes: Seq[AbstractDataType] = Seq(DecimalType, StringType)
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val inputTypeCheck = super.checkInputDataTypes()
+    if (inputTypeCheck.isSuccess) {
+      if (right.foldable) {
+        numberFormatter.check()
+      } else {
+        TypeCheckResult.TypeCheckFailure(s"Format expression must be foldable, 
but got $right")
+      }
+    } else {
+      inputTypeCheck
+    }
+  }
+  override def prettyName: String = "to_char"
+  override def nullSafeEval(decimal: Any, format: Any): Any = {
+    val input = decimal.asInstanceOf[Decimal]
+    numberFormatter.format(input)
+  }
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+    val builder =
+      ctx.addReferenceObj("builder", numberFormatter, 
classOf[ToNumberParser].getName)
+    val eval = left.genCode(ctx)
+    val result =
+      code"""
+         |${eval.code}
+         |boolean ${ev.isNull} = ${eval.isNull};
+         |${CodeGenerator.javaType(dataType)} ${ev.value} = 
${CodeGenerator.defaultValue(dataType)};
+         |if (!${ev.isNull}) {
+         |  ${ev.value} = $builder.format(${eval.value});
+         |}
+      """
+    val stripped = result.stripMargin
+    ev.copy(code = stripped)
+  }
+  override protected def withNewChildrenInternal(
+      newLeft: Expression, newRight: Expression): ToCharacter =
+    copy(left = newLeft, right = newRight)
+}
+
+/**
+ * A function that converts decimal values to strings, returning NULL if the 
decimal value fails to
+ * match the format string.
+ */
+@ExpressionDescription(
+  usage = """
+
+    _FUNC_(numberExpr, formatExpr) - Convert `numberExpr` to a string based on 
the `formatExpr`.
+      Returns NULL if the conversion fails. The format follows the same 
semantics as the
+      to_char function.
+  """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_(454, '999');
+       454
+      > SELECT _FUNC_(454.00, '000D00');
+       454.00
+      > SELECT _FUNC_(12454, '99G999');
+       12,454
+      > SELECT _FUNC_(78.12, '$99.99');
+       $78.12
+      > SELECT _FUNC_(-12454.8, '99G999D9S');
+       12,454.8-
+  """,
+  since = "3.4.0",
+  group = "string_funcs")
+case class TryToCharacter(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant {
+  private lazy val numberFormat = 
right.eval().toString.toUpperCase(Locale.ROOT)
+  private lazy val numberFormatter = new ToNumberParser(numberFormat, false)
+
+  override def dataType: DataType = StringType
+  override def inputTypes: Seq[AbstractDataType] = Seq(DecimalType, StringType)
+  override def nullable: Boolean = true
+  override def checkInputDataTypes(): TypeCheckResult =
+    ToCharacter(left, right).checkInputDataTypes()
+  override def prettyName: String = "try_to_char"
+  override def nullSafeEval(decimal: Any, format: Any): Any = {
+    val input = decimal.asInstanceOf[Decimal]
+    numberFormatter.format(input)
+  }
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+    val builder =
+      ctx.addReferenceObj("builder", numberFormatter, 
classOf[ToNumberParser].getName)
+    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}) {
+         |  UTF8String result = $builder.format(${eval.value});
+         |  if (result == null) {
+         |    ${ev.isNull} = true;
+         |    ${ev.value} = null;
+         |  } else {
+         |    ${ev.isNull} = false;
+         |    ${ev.value} = result;
+         |  }
+         |}
+      """.stripMargin)
+  }
+
+  override protected def withNewChildrenInternal(
+      newLeft: Expression,
+      newRight: Expression): TryToCharacter =
+    copy(left = newLeft, right = newRight)
+}
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala
index 716224983e0..f5c791e0105 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala
@@ -37,6 +37,8 @@ object ToNumberParser {
   final val PLUS_SIGN = '+'
   final val POINT_LETTER = 'D'
   final val POINT_SIGN = '.'
+  final val POUND_SIGN = '#'
+  final val SPACE = ' '
   final val ZERO_DIGIT = '0'
 
   final val OPTIONAL_MINUS_STRING = "MI"
@@ -235,13 +237,16 @@ class ToNumberParser(numberFormat: String, errorOnFail: 
Boolean) extends Seriali
   }
 
   // Holds all digits (0-9) before the decimal point (.) while parsing each 
input string.
-  private lazy val beforeDecimalPoint = new StringBuilder(precision)
+  private lazy val parsedBeforeDecimalPoint = new StringBuilder(precision)
   // Holds all digits (0-9) after the decimal point (.) while parsing each 
input string.
-  private lazy val afterDecimalPoint = new StringBuilder(scale)
+  private lazy val parsedAfterDecimalPoint = new StringBuilder(scale)
   // Number of digits (0-9) in each group of the input string, split by 
thousands separators.
   private lazy val parsedDigitGroupSizes = mutable.Buffer.empty[Int]
   // Increments to count the number of digits (0-9) in the current group 
within the input string.
-  private var numDigitsInCurrentGroup: Int = 0
+  private var parsedNumDigitsInCurrentGroup: Int = 0
+  // These are indexes into the characters of the input string before and 
after the decimal point.
+  private var formattingBeforeDecimalPointIndex = 0
+  private var formattingAfterDecimalPointIndex = 0
 
   /**
    * The result type of this parsing is a Decimal value with the appropriate 
precision and scale.
@@ -320,7 +325,7 @@ class ToNumberParser(numberFormat: String, errorOnFail: 
Boolean) extends Seriali
             false
         })
     }) {
-      return "Thousands separators (,) must have digits in between them " +
+      return "Thousands separators (, or G) must have digits in between them " 
+
         s"in the number format: '$numberFormat'"
     }
     // Make sure that thousands separators does not appear after the decimal 
point, if any.
@@ -328,7 +333,7 @@ class ToNumberParser(numberFormat: String, errorOnFail: 
Boolean) extends Seriali
       case DigitGroups(tokens, digits) =>
         tokens.length > digits.length
     }) {
-      return "Thousands separators (,) may not appear after the decimal point 
" +
+      return "Thousands separators (, or G) may not appear after the decimal 
point " +
         s"in the number format: '$numberFormat'"
     }
     // Make sure that the format string does not contain any prohibited 
duplicate tokens.
@@ -394,8 +399,9 @@ class ToNumberParser(numberFormat: String, errorOnFail: 
Boolean) extends Seriali
     val inputString = input.toString
     val inputLength = inputString.length
     // Build strings representing all digits before and after the decimal 
point, respectively.
-    beforeDecimalPoint.clear()
-    afterDecimalPoint.clear()
+    parsedBeforeDecimalPoint.clear()
+    parsedAfterDecimalPoint.clear()
+    // Tracks whether we've reached the decimal point yet in either parsing or 
formatting.
     var reachedDecimalPoint = false
     // Record whether the input specified a negative result, such as with a 
minus sign.
     var negateResult = false
@@ -471,7 +477,7 @@ class ToNumberParser(numberFormat: String, errorOnFail: 
Boolean) extends Seriali
       // in the input string, then the input string does not match the format 
string.
       formatMatchFailure(input, numberFormat)
     } else {
-      getDecimal(negateResult)
+      parseResultToDecimalValue(negateResult)
     }
   }
 
@@ -493,16 +499,16 @@ class ToNumberParser(numberFormat: String, errorOnFail: 
Boolean) extends Seriali
     val inputLength = inputString.length
     // Consume characters from the current input index forwards in the input 
string as long as
     // they are digits (0-9) or the thousands separator (,).
-    numDigitsInCurrentGroup = 0
+    parsedNumDigitsInCurrentGroup = 0
     var inputIndex = startingInputIndex
     parsedDigitGroupSizes.clear()
 
     while (inputIndex < inputLength &&
-      matchesDigitOrComma(inputString(inputIndex), reachedDecimalPoint)) {
+      parsedCharMatchesDigitOrComma(inputString(inputIndex), 
reachedDecimalPoint)) {
       inputIndex += 1
     }
     if (inputIndex == inputLength) {
-      parsedDigitGroupSizes.prepend(numDigitsInCurrentGroup)
+      parsedDigitGroupSizes.prepend(parsedNumDigitsInCurrentGroup)
     }
     // Compare the number of digits encountered in each group (separated by 
thousands
     // separators) with the expected numbers from the format string.
@@ -537,27 +543,27 @@ class ToNumberParser(numberFormat: String, errorOnFail: 
Boolean) extends Seriali
    * Returns true if the given character matches a digit (0-9) or a comma, 
updating fields of
    * this class related to parsing during the process.
    */
-  private def matchesDigitOrComma(char: Char, reachedDecimalPoint: Boolean): 
Boolean = {
+  private def parsedCharMatchesDigitOrComma(char: Char, reachedDecimalPoint: 
Boolean): Boolean = {
     char match {
       case _ if char.isWhitespace =>
         // Ignore whitespace and keep advancing through the input string.
         true
-      case _ if char >= ZERO_DIGIT && char <= NINE_DIGIT =>
-        numDigitsInCurrentGroup += 1
-        // Append each group of input digits to the appropriate 
before/afterDecimalPoint
+      case _ if char.isDigit =>
+        parsedNumDigitsInCurrentGroup += 1
+        // Append each group of input digits to the appropriate 
before/parsedAfterDecimalPoint
         // string for later use in constructing the result Decimal value.
         if (reachedDecimalPoint) {
-          afterDecimalPoint.append(char)
+          parsedAfterDecimalPoint.append(char)
         } else {
-          beforeDecimalPoint.append(char)
+          parsedBeforeDecimalPoint.append(char)
         }
         true
       case COMMA_SIGN =>
-        parsedDigitGroupSizes.prepend(numDigitsInCurrentGroup)
-        numDigitsInCurrentGroup = 0
+        parsedDigitGroupSizes.prepend(parsedNumDigitsInCurrentGroup)
+        parsedNumDigitsInCurrentGroup = 0
         true
       case _ =>
-        parsedDigitGroupSizes.prepend(numDigitsInCurrentGroup)
+        parsedDigitGroupSizes.prepend(parsedNumDigitsInCurrentGroup)
         false
     }
   }
@@ -568,28 +574,37 @@ class ToNumberParser(numberFormat: String, errorOnFail: 
Boolean) extends Seriali
    */
   private def formatMatchFailure(input: UTF8String, originNumberFormat: 
String): Decimal = {
     if (errorOnFail) {
-      throw QueryExecutionErrors.invalidNumberFormatError(input, 
originNumberFormat)
+      throw QueryExecutionErrors.invalidNumberFormatError(
+        "string", input.toString, originNumberFormat)
+    }
+    null
+  }
+  private def formatMatchFailure(input: Decimal, originNumberFormat: String): 
UTF8String = {
+    if (errorOnFail) {
+      throw QueryExecutionErrors.invalidNumberFormatError(
+        "Decimal value", input.toString, originNumberFormat)
     }
     null
   }
 
   /**
-   * Computes the final Decimal value from the beforeDecimalPoint and 
afterDecimalPoint fields of
-   * this class, as a result of parsing.
+   * Computes the final Decimal value from the parsedBeforeDecimalPoint and 
parsedAfterDecimalPoint
+   * fields of this class, as a result of parsing.
    *
    * @param negateResult whether the input string specified to negate the 
result
    * @return a Decimal value with the value indicated by the input string and 
the precision and
    *         scale indicated by the format string
    */
-  private def getDecimal(negateResult: Boolean): Decimal = {
-    // Append zeros to the afterDecimalPoint until it comprises the same 
number of digits as the
-    // scale. This is necessary because we must determine the scale from the 
format string alone but
-    // each input string may include a variable number of digits after the 
decimal point.
-    val extraZeros = "0" * (scale - afterDecimalPoint.length)
-    val afterDecimalPadded = afterDecimalPoint.toString + extraZeros
+  private def parseResultToDecimalValue(negateResult: Boolean): Decimal = {
+    // Append zeros to the parsedAfterDecimalPoint string until it comprises 
the same number of
+    // digits as the scale. This is necessary because we must determine the 
scale from the format
+    // string alone but each input string may include a variable number of 
digits after the decimal
+    // point.
+    val extraZeros = "0" * (scale - parsedAfterDecimalPoint.length)
+    val afterDecimalPadded = parsedAfterDecimalPoint.toString + extraZeros
     val prefix = if (negateResult) "-" else ""
     val suffix = if (afterDecimalPadded.nonEmpty) "." + afterDecimalPadded 
else ""
-    val numStr = s"$prefix$beforeDecimalPoint$suffix"
+    val numStr = s"$prefix$parsedBeforeDecimalPoint$suffix"
     val javaDecimal = new java.math.BigDecimal(numStr)
     if (precision <= Decimal.MAX_LONG_DIGITS) {
       // Constructs a `Decimal` with an unscaled `Long` value if possible.
@@ -599,4 +614,249 @@ class ToNumberParser(numberFormat: String, errorOnFail: 
Boolean) extends Seriali
       Decimal(javaDecimal, precision, scale)
     }
   }
+
+  /**
+   * Converts a decimal value to a string based on the given number format.
+   *
+   * Iterates through the [[formatTokens]] obtained from processing the format 
string, while also
+   * inspecting the input decimal value.
+   *
+   * @param input the decimal value that needs to be converted
+   * @return the result String value obtained from string formatting
+   */
+  def format(input: Decimal): UTF8String = {
+    val result = new StringBuilder()
+    // These are string representations of the input Decimal value.
+    val (inputBeforeDecimalPoint: String,
+      inputAfterDecimalPoint: String) =
+      formatSplitInputBeforeAndAfterDecimalPoint(input)
+    // These are indexes into the characters of the input string before and 
after the decimal point.
+    formattingBeforeDecimalPointIndex = 0
+    formattingAfterDecimalPointIndex = 0
+    var reachedDecimalPoint = false
+
+    // Iterate through the tokens representing the provided format string, in 
order.
+    for (formatToken: InputToken <- formatTokens) {
+      formatToken match {
+        case groups: DigitGroups =>
+          formatDigitGroups(
+            groups, inputBeforeDecimalPoint, inputAfterDecimalPoint, 
reachedDecimalPoint, result)
+        case DecimalPoint() =>
+          // If the last character so far is a space, change it to a zero. 
This means the input
+          // decimal does not have an integer part.
+          if (result.nonEmpty && result.last == SPACE) {
+            result(result.length - 1) = ZERO_DIGIT
+          }
+          result.append(POINT_SIGN)
+          reachedDecimalPoint = true
+        case DollarSign() =>
+          result.append(DOLLAR_SIGN)
+        case _: OptionalPlusOrMinusSign =>
+          stripTrailingLoneDecimalPoint(result)
+          if (input < Decimal.ZERO) {
+            addCharacterCheckingTrailingSpaces(result, MINUS_SIGN)
+          } else {
+            addCharacterCheckingTrailingSpaces(result, PLUS_SIGN)
+          }
+        case _: OptionalMinusSign =>
+          if (input < Decimal.ZERO) {
+            stripTrailingLoneDecimalPoint(result)
+            addCharacterCheckingTrailingSpaces(result, MINUS_SIGN)
+          } else {
+            result.append(SPACE)
+          }
+        case OpeningAngleBracket() =>
+          if (input < Decimal.ZERO) {
+            result.append(ANGLE_BRACKET_OPEN)
+          }
+        case ClosingAngleBracket() =>
+          stripTrailingLoneDecimalPoint(result)
+          if (input < Decimal.ZERO) {
+            addCharacterCheckingTrailingSpaces(result, ANGLE_BRACKET_CLOSE)
+          } else {
+            result.append(SPACE)
+            result.append(SPACE)
+          }
+      }
+    }
+
+    if (formattingBeforeDecimalPointIndex < inputBeforeDecimalPoint.length ||
+      formattingAfterDecimalPointIndex < inputAfterDecimalPoint.length) {
+      // Remaining digits before or after the decimal point exist in the 
decimal value but not in
+      // the format string.
+      formatMatchFailure(input, numberFormat)
+    } else {
+      stripTrailingLoneDecimalPoint(result)
+      val str = result.toString
+      if (result.isEmpty || str == "+" || str == "-") {
+        UTF8String.fromString("0")
+      } else {
+        UTF8String.fromString(str)
+      }
+    }
+  }
+
+  /**
+   * Splits the provided Decimal value's string representation by the decimal 
point, if any.
+   * @param input the Decimal value to consume
+   * @return two strings representing the contents before and after the 
decimal point (if any)
+   */
+  private def formatSplitInputBeforeAndAfterDecimalPoint(input: Decimal): 
(String, String) = {
+    // Convert the input Decimal value to a string (without exponent notation).
+    val inputString = input.toJavaBigDecimal.toPlainString
+    // Split the digits before and after the decimal point.
+    val tokens: Array[String] = inputString.split(POINT_SIGN)
+    var beforeDecimalPoint: String = tokens(0)
+    var afterDecimalPoint: String = if (tokens.length > 1) tokens(1) else ""
+    // Strip any leading minus sign to consider the digits only.
+    // Strip leading and trailing zeros to match cases when the format string 
begins with a decimal
+    // point.
+    beforeDecimalPoint = beforeDecimalPoint.dropWhile(c => c == MINUS_SIGN || 
c == ZERO_DIGIT)
+    afterDecimalPoint = afterDecimalPoint.reverse.dropWhile(_ == 
ZERO_DIGIT).reverse
+
+    // If the format string specifies more digits than the 
'beforeDecimalPoint', prepend leading
+    // spaces to make them the same length. Likewise, if the format string 
specifies more digits
+    // than the 'afterDecimalPoint', append trailing spaces to make them the 
same length. This step
+    // simplifies logic consuming the format tokens later.
+    var reachedDecimalPoint = false
+    var numFormatDigitsBeforeDecimalPoint = 0
+    var numFormatDigitsAfterDecimalPoint = 0
+    formatTokens.foreach {
+      case digitGroups: DigitGroups =>
+        digitGroups.digits.foreach { digits =>
+          val numDigits = digits match {
+            case ExactlyAsManyDigits(num) => num
+            case AtMostAsManyDigits(num) => num
+          }
+          for (_ <- 0 until numDigits) {
+            if (!reachedDecimalPoint) {
+              numFormatDigitsBeforeDecimalPoint += 1
+            } else {
+              numFormatDigitsAfterDecimalPoint += 1
+            }
+          }
+        }
+      case _: DecimalPoint =>
+        reachedDecimalPoint = true
+      case _ =>
+    }
+    // If there were more digits in the provided input string (before or after 
the decimal point)
+    // than specified in the format string, this is an overflow.
+    if (numFormatDigitsBeforeDecimalPoint < beforeDecimalPoint.length ||
+      numFormatDigitsAfterDecimalPoint < afterDecimalPoint.length) {
+      beforeDecimalPoint = "#" * numFormatDigitsBeforeDecimalPoint
+      afterDecimalPoint = "#" * numFormatDigitsAfterDecimalPoint
+    }
+    val leadingSpaces = " " * (numFormatDigitsBeforeDecimalPoint - 
beforeDecimalPoint.length)
+    val trailingZeros = "0" * (numFormatDigitsAfterDecimalPoint - 
afterDecimalPoint.length)
+    (leadingSpaces + beforeDecimalPoint, afterDecimalPoint + trailingZeros)
+  }
+
+  /**
+   * Performs format processing on the digits in [[groups]], updating 
[[result]].
+   *
+   * @param groups the token representing a group of digits from the format 
string
+   * @param inputBeforeDecimalPoint string representation of the input decimal 
value before the
+   *                                decimal point
+   * @param inputAfterDecimalPoint string representation of the input decimal 
value after the
+   *                               decimal point
+   * @param reachedDecimalPoint true if we have reached the decimal point so 
far during processing
+   * @param result the result of formatting is built here as a string during 
iteration
+   */
+  private def formatDigitGroups(
+      groups: DigitGroups,
+      inputBeforeDecimalPoint: String,
+      inputAfterDecimalPoint: String,
+      reachedDecimalPoint: Boolean,
+      result: StringBuilder): Unit = {
+    // Iterate through the tokens in the DigitGroups. Reverse the order of the 
tokens so we
+    // consume them in the left-to-right order that they originally appeared 
in the format
+    // string.
+    for (digitGroupToken <- groups.tokens.reverse) {
+      digitGroupToken match {
+        case digits: Digits if !reachedDecimalPoint =>
+          val numDigits = digits match {
+            case ExactlyAsManyDigits(num) => num
+            case AtMostAsManyDigits(num) => num
+          }
+          for (_ <- 0 until numDigits) {
+            inputBeforeDecimalPoint(formattingBeforeDecimalPointIndex) match {
+              case SPACE if digits.isInstanceOf[ExactlyAsManyDigits] =>
+                // The format string started with a zero and had more digits 
than the provided
+                // input string, so we prepend a zero to the result. Note that 
there is no need to
+                // check for the presence of any previous positive or minus 
sign in the result
+                // because we are adding zeros here and we want them to go 
directly after such a
+                // sign, such as "-00000123.45".
+                result.append(ZERO_DIGIT)
+              case SPACE =>
+                addSpaceCheckingTrailingCharacters(result)
+              case c: Char =>
+                result.append(c)
+            }
+            formattingBeforeDecimalPointIndex += 1
+          }
+        case digits: Digits if reachedDecimalPoint =>
+          val numDigits = digits match {
+            case ExactlyAsManyDigits(num) => num
+            case AtMostAsManyDigits(num) => num
+          }
+          for (_ <- 0 until numDigits) {
+            
result.append(inputAfterDecimalPoint(formattingAfterDecimalPointIndex))
+            formattingAfterDecimalPointIndex += 1
+          }
+        case _: ThousandsSeparator =>
+          if (result.nonEmpty && result.last.isDigit) {
+            result.append(COMMA_SIGN)
+          } else {
+            addSpaceCheckingTrailingCharacters(result)
+          }
+      }
+    }
+  }
+
+  /**
+   * Adds a character to the end of the string builder. After doing so, if we 
just added the
+   * character after a space, swap the characters.
+   */
+  private def addCharacterCheckingTrailingSpaces(result: StringBuilder, char: 
Char): Unit = {
+    result.append(char)
+    var i = result.size - 1
+    while (i >= 1 &&
+      result(i - 1) == SPACE &&
+      result(i) == char) {
+      result(i) = SPACE
+      result(i - 1) = char
+      i -= 1
+    }
+  }
+
+  /**
+   * Adds a character to the end of the string builder. After doing so, if we 
just added the
+   * character after cases like unary plus or minus, swap the characters.
+   */
+  private def addSpaceCheckingTrailingCharacters(result: StringBuilder): Unit 
= {
+    result.append(SPACE)
+    var i = result.size - 1
+    while (i >= 1 &&
+      (result(i - 1) == PLUS_SIGN ||
+        result(i - 1) == MINUS_SIGN ||
+        result(i - 1) == ANGLE_BRACKET_OPEN) &&
+      result(i) == SPACE) {
+      result(i) = result(i - 1)
+      result(i - 1) = SPACE
+      i -= 1
+    }
+  }
+
+  /**
+   * If the result string ends with a decimal point, strip it.
+   */
+  private def stripTrailingLoneDecimalPoint(result: StringBuilder): Unit = {
+    val i = result.indexOf(POINT_SIGN)
+    if (i != -1 &&
+      (i == result.length - 1 ||
+        result(i + 1) == SPACE)) {
+      result(i) = SPACE
+    }
+  }
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index f9d3854fc5e..447a820a128 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -1959,9 +1959,9 @@ object QueryExecutionErrors extends QueryErrorsBase {
         s" to at least $numWrittenParts.")
   }
 
-  def invalidNumberFormatError(input: UTF8String, format: String): Throwable = 
{
+  def invalidNumberFormatError(valueType: String, input: String, format: 
String): Throwable = {
     new IllegalArgumentException(
-      s"The input string '$input' does not match the given number format: 
'$format'")
+      s"The input $valueType '$input' does not match the given number format: 
'$format'")
   }
 
   def multipleBucketTransformsError(): Throwable = {
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
index b37d8ca177f..2c184f29b8f 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala
@@ -1007,11 +1007,11 @@ class StringExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
     }
   }
 
-  test("ToNumber: negative tests (the format string is invalid)") {
+  test("ToNumber and ToCharacter: negative tests (the format string is 
invalid)") {
     val unexpectedCharacter = "the structure of the format string must match: 
" +
       "[MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] [PR|MI|S]"
     val thousandsSeparatorDigitsBetween =
-      "Thousands separators (,) must have digits in between them"
+      "Thousands separators (, or G) must have digits in between them"
     val mustBeAtEnd = "must be at the end of the number format"
     val atMostOne = "At most one"
     Seq(
@@ -1049,7 +1049,8 @@ class StringExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
       (",,345", "9,99,") -> thousandsSeparatorDigitsBetween,
       (",,345", ",,999,099.99") -> thousandsSeparatorDigitsBetween,
       // Thousands separators must not appear after the decimal point.
-      ("123.45,6", "099.99,9") -> "Thousands separators (,) may not appear 
after the decimal point"
+      ("123.45,6", "099.99,9") ->
+        "Thousands separators (, or G) may not appear after the decimal point"
     ).foreach { case ((str: String, format: String), expectedErrMsg: String) =>
       val toNumberResult = ToNumber(Literal(str), 
Literal(format)).checkInputDataTypes()
       assert(toNumberResult != TypeCheckResult.TypeCheckSuccess,
@@ -1066,6 +1067,22 @@ class StringExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
         case TypeCheckResult.TypeCheckFailure(message) =>
           assert(message.contains(expectedErrMsg))
       }
+
+      val toCharResult = ToCharacter(Decimal(456), 
Literal(format)).checkInputDataTypes()
+      assert(toCharResult != TypeCheckResult.TypeCheckSuccess,
+        s"The format string should have been invalid: $format")
+      toCharResult match {
+        case TypeCheckResult.TypeCheckFailure(message) =>
+          assert(message.contains(expectedErrMsg))
+      }
+
+      val tryToCharResult = TryToCharacter(Decimal(456), 
Literal(format)).checkInputDataTypes()
+      assert(tryToCharResult != TypeCheckResult.TypeCheckSuccess,
+        s"The format string should have been invalid: $format")
+      tryToCharResult match {
+        case TypeCheckResult.TypeCheckFailure(message) =>
+          assert(message.contains(expectedErrMsg))
+      }
     }
   }
 
@@ -1108,6 +1125,366 @@ class StringExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
     }
   }
 
+  test("ToCharacter: positive tests") {
+    // Test '0' and '9'
+    Seq(
+      (Decimal(454),
+        "9999") ->
+        " 454",
+      (Decimal(454),
+        "99999") ->
+        "  454",
+      (Decimal(4),
+        "0") ->
+        "4",
+      (Decimal(45),
+        "00") ->
+        "45",
+      (Decimal(454),
+        "000") ->
+        "454",
+      (Decimal(454),
+        "0000") ->
+        "0454",
+      (Decimal(454),
+        "00000") ->
+        "00454"
+    ).foreach { case ((decimal, format), expected) =>
+      var expr: Expression = ToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = TryToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+    }
+
+    // Test '.' and 'D'
+    Seq(
+      (Decimal(0.4542),
+        ".00000") ->
+        ".45420",
+      (Decimal(454.2),
+        "000.0") ->
+        "454.2",
+      (Decimal(454),
+        "000.0") ->
+        "454.0",
+      (Decimal(454.2),
+        "000.00") ->
+        "454.20",
+      (Decimal(454),
+        "000.00") ->
+        "454.00",
+      (Decimal(0.4542),
+        ".0000") ->
+        ".4542",
+      (Decimal(4542),
+        "0000.") ->
+        "4542 "
+    ).foreach { case ((decimal, format), expected) =>
+      val format2 = format.replace('.', 'D')
+      var expr: Expression = ToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = ToCharacter(Literal(decimal), Literal(format2))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = TryToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = TryToCharacter(Literal(decimal), Literal(format2))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+    }
+
+    Seq(
+      (Decimal(454.2),
+        "0000.00") ->
+        "0454.20",
+      (Decimal(454),
+        "0000.00") ->
+        "0454.00",
+      (Decimal(4542),
+        "00000.") ->
+        "04542 ",
+      (Decimal(454.2),
+        "9999.99") ->
+        " 454.20",
+      (Decimal(454),
+        "9999.99") ->
+        " 454.00",
+      // There are no digits after the decimal point.
+      (Decimal(4542),
+        "99999.") ->
+        " 4542 "
+    ).foreach { case ((decimal, format), expected) =>
+      var expr: Expression = ToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = TryToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+    }
+
+    // Test ',' and 'G'
+    Seq(
+      (Decimal(12454),
+        "0,0000") ->
+        "1,2454",
+      (Decimal(12454),
+        "00,000") ->
+        "12,454",
+      (Decimal(124543),
+        "000,000") ->
+        "124,543",
+      (Decimal(12),
+        "000,000") ->
+        "000,012",
+      (Decimal(1245436),
+        "0,000,000") ->
+        "1,245,436",
+      (Decimal(12454367),
+        "00,000,000") ->
+        "12,454,367"
+    ).foreach { case ((decimal, format), expected) =>
+      val format2 = format.replace(',', 'G')
+      var expr: Expression = ToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = ToCharacter(Literal(decimal), Literal(format2))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = TryToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = TryToCharacter(Literal(decimal), Literal(format2))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+    }
+
+    Seq(
+      (Decimal(12454),
+        "000,000") ->
+        "012,454",
+      (Decimal(12454),
+        "00,0000") ->
+        "01,2454",
+      (Decimal(12454),
+        "000,0000") ->
+        "001,2454",
+      (Decimal(12454),
+        "0000,0000") ->
+        "0001,2454",
+      (Decimal(12454),
+        "00,0000") ->
+        "01,2454",
+      (Decimal(12454),
+        "000,0000") ->
+        "001,2454",
+      (Decimal(12454),
+        "0000,0000") ->
+        "0001,2454",
+      (Decimal(12454367),
+        "000,000,000") ->
+        "012,454,367",
+      (Decimal(12454),
+        "999,999") ->
+        " 12,454",
+      (Decimal(12454),
+        "9,9999") ->
+        "1,2454",
+      (Decimal(12454),
+        "99,9999") ->
+        " 1,2454",
+      (Decimal(12454),
+        "999,9999") ->
+        "  1,2454",
+      (Decimal(12454),
+        "9999,9999") ->
+        "   1,2454",
+      (Decimal(12454367),
+        "999,999,999") ->
+        " 12,454,367",
+      (Decimal(12454),
+        "999,999") ->
+        " 12,454"
+    ).foreach { case ((decimal, format), expected) =>
+      var expr: Expression = ToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = TryToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+    }
+
+    // Test '$'
+    Seq(
+      (Decimal(78.12),
+        "$99.99") ->
+        "$78.12",
+      (Decimal(78.12),
+        "$00.00") ->
+        "$78.12"
+    ).foreach { case ((decimal, format), expected) =>
+      var expr: Expression = ToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = TryToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+    }
+
+    // Test 'S'
+    Seq(
+      (Decimal(83028485),
+        "S99999999999.9999999") ->
+        "   +83028485.0000000",
+      (Decimal(0),
+        "9999999999999999.999999999999999S") ->
+        "               0.000000000000000+",
+      (Decimal(unscaled = 43100000000L, precision = 38, scale = 10),
+        "9999999999999999.999999999999999S") ->
+        "               4.310000000000000+",
+      (Decimal(-454.8),
+        "99G999.9S") ->
+        "   454.8-",
+      (Decimal(-454.8),
+        "00G000.0S") ->
+        "00,454.8-",
+      (Decimal(-454),
+        "S999") ->
+        "-454",
+      (Decimal(-454),
+        "999S") ->
+        "454-",
+      (Decimal(-12454.8),
+        "99G999D9S") ->
+        "12,454.8-",
+      (Decimal(-83028485),
+        "99999999999.9999999S") ->
+        "   83028485.0000000-"
+    ).foreach { case ((decimal, format), expected) =>
+      var expr: Expression = ToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = TryToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+    }
+
+    // Test 'MI'
+    Seq(
+      (Decimal(4.31),
+        "9999999999999999.999999999999999MI") ->
+        "               4.310000000000000 ",
+      (Decimal(0),
+        "9999999999999999.999999999999999MI") ->
+        "               0.000000000000000 ",
+      (Decimal(unscaled = 43100000000L, precision = 38, scale = 10),
+        "9999999999999999.999999999999999MI") ->
+        "               4.310000000000000 ",
+      (Decimal(-454.8),
+        "99G999.9MI") ->
+        "   454.8-",
+      (Decimal(-454.8),
+        "00G000.0MI") ->
+        "00,454.8-",
+      (Decimal(-454),
+        "999MI") ->
+        "454-",
+      (Decimal(-12454.8),
+        "99G999D9MI") ->
+        "12,454.8-",
+      (Decimal(-4.31),
+        "MI9999999999999999.999999999999999") ->
+        "               -4.310000000000000"
+    ).foreach { case ((decimal, format), expected) =>
+      var expr: Expression = ToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = TryToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+    }
+
+    // Test 'PR'
+    Seq(
+      (Decimal(4.31),
+        "9999999999999999.999999999999999PR") ->
+        "               4.310000000000000  ",
+      (Decimal(0),
+        "9999999999999999.999999999999999PR") ->
+        "               0.000000000000000  ",
+      (Decimal(unscaled = 43100000000L, precision = 38, scale = 10),
+        "9999999999999999.999999999999999PR") ->
+        "               4.310000000000000  ",
+      (Decimal(-123),
+        "9999999999999999.999PR") ->
+        "             <123.000>",
+      (Decimal(-123.4),
+        "9999999999999999.999PR") ->
+        "             <123.400>",
+      (Decimal(-454.8),
+        "99G999.9PR") ->
+        "   <454.8>",
+      (Decimal(-454.8),
+        "00G000.0PR") ->
+        "<00,454.8>",
+      (Decimal(-454),
+        "999PR") ->
+        "<454>",
+      (Decimal(-12454.8),
+        "99G999D9PR") ->
+        "<12,454.8>"
+    ).foreach { case ((decimal, format), expected) =>
+      var expr: Expression = ToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = TryToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+    }
+
+    // Test overflows
+    Seq(
+      // If there were more digits in the provided input string (before or 
after the decimal point)
+      // than specified in the format string, an overflow takes place.
+      (Decimal(454),
+        "0") ->
+        "#",
+      (Decimal(454),
+        "00") ->
+        "##",
+      (Decimal(4.67),
+        "9.9") ->
+        "#.#",
+      (Decimal(4.67),
+        "99.9") ->
+        "##.#"
+    ).foreach { case ((decimal, format), expected) =>
+      var expr: Expression = ToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+
+      expr = TryToCharacter(Literal(decimal), Literal(format))
+      assert(expr.checkInputDataTypes() == TypeCheckResult.TypeCheckSuccess)
+      checkEvaluation(expr, expected)
+    }
+  }
+
   test("find in set") {
     checkEvaluation(
       FindInSet(Literal.create(null, StringType), Literal.create(null, 
StringType)), null)
diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md 
b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
index 618e1e8087c..93c12487200 100644
--- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
+++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
@@ -298,6 +298,7 @@
 | org.apache.spark.sql.catalyst.expressions.Tanh | tanh | SELECT tanh(0) | 
struct<TANH(0):double> |
 | org.apache.spark.sql.catalyst.expressions.TimeWindow | window | SELECT a, 
window.start, window.end, count(*) as cnt FROM VALUES ('A1', '2021-01-01 
00:00:00'), ('A1', '2021-01-01 00:04:30'), ('A1', '2021-01-01 00:06:00'), 
('A2', '2021-01-01 00:01:00') AS tab(a, b) GROUP by a, window(b, '5 minutes') 
ORDER BY a, start | struct<a:string,start:timestamp,end:timestamp,cnt:bigint> |
 | org.apache.spark.sql.catalyst.expressions.ToBinary | to_binary | SELECT 
to_binary('abc', 'utf-8') | struct<to_binary(abc, utf-8):binary> |
+| org.apache.spark.sql.catalyst.expressions.ToCharacter | to_char | SELECT 
to_char(454, '999') | struct<to_char(454, 999):string> |
 | org.apache.spark.sql.catalyst.expressions.ToDegrees | degrees | SELECT 
degrees(3.141592653589793) | struct<DEGREES(3.141592653589793):double> |
 | org.apache.spark.sql.catalyst.expressions.ToNumber | to_number | SELECT 
to_number('454', '999') | struct<to_number(454, 999):decimal(3,0)> |
 | org.apache.spark.sql.catalyst.expressions.ToRadians | radians | SELECT 
radians(180) | struct<RADIANS(180):double> |
@@ -313,6 +314,7 @@
 | org.apache.spark.sql.catalyst.expressions.TryMultiply | try_multiply | 
SELECT try_multiply(2, 3) | struct<try_multiply(2, 3):int> |
 | org.apache.spark.sql.catalyst.expressions.TrySubtract | try_subtract | 
SELECT try_subtract(2, 1) | struct<try_subtract(2, 1):int> |
 | org.apache.spark.sql.catalyst.expressions.TryToBinary | try_to_binary | 
SELECT try_to_binary('abc', 'utf-8') | struct<try_to_binary(abc, utf-8):binary> 
|
+| org.apache.spark.sql.catalyst.expressions.TryToCharacter | try_to_char | 
SELECT try_to_char(454, '999') | struct<try_to_char(454, 999):string> |
 | org.apache.spark.sql.catalyst.expressions.TryToNumber | try_to_number | 
SELECT try_to_number('454', '999') | struct<try_to_number(454, 
999):decimal(3,0)> |
 | org.apache.spark.sql.catalyst.expressions.TypeOf | typeof | SELECT typeof(1) 
| struct<typeof(1):string> |
 | org.apache.spark.sql.catalyst.expressions.UnBase64 | unbase64 | SELECT 
unbase64('U3BhcmsgU1FM') | struct<unbase64(U3BhcmsgU1FM):binary> |
diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql 
b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql
index 5fea758e730..28be31f9048 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql
@@ -101,34 +101,33 @@ SELECT q2, abs(q2) FROM INT8_TBL;
 SELECT min(q1), min(q2) FROM INT8_TBL;
 SELECT max(q1), max(q2) FROM INT8_TBL;
 
--- [SPARK-28137] Missing Data Type Formatting Functions
 -- TO_CHAR()
---
--- SELECT '' AS to_char_1, to_char(q1, '9G999G999G999G999G999'), to_char(q2, 
'9,999,999,999,999,999')
---     FROM INT8_TBL;
-
--- SELECT '' AS to_char_2, to_char(q1, '9G999G999G999G999G999D999G999'), 
to_char(q2, '9,999,999,999,999,999.999,999')
---     FROM INT8_TBL;
-
--- SELECT '' AS to_char_3, to_char( (q1 * -1), '9999999999999999PR'), to_char( 
(q2 * -1), '9999999999999999.999PR')
---     FROM INT8_TBL;
-
--- SELECT '' AS to_char_4, to_char( (q1 * -1), '9999999999999999S'), to_char( 
(q2 * -1), 'S9999999999999999')
---     FROM INT8_TBL;
-
--- SELECT '' AS to_char_5,  to_char(q2, 'MI9999999999999999')     FROM 
INT8_TBL;
--- SELECT '' AS to_char_6,  to_char(q2, 'FMS9999999999999999')    FROM 
INT8_TBL;
--- SELECT '' AS to_char_7,  to_char(q2, 'FM9999999999999999THPR') FROM 
INT8_TBL;
--- SELECT '' AS to_char_8,  to_char(q2, 'SG9999999999999999th')   FROM 
INT8_TBL;
--- SELECT '' AS to_char_9,  to_char(q2, '0999999999999999')       FROM 
INT8_TBL;
--- SELECT '' AS to_char_10, to_char(q2, 'S0999999999999999')      FROM 
INT8_TBL;
--- SELECT '' AS to_char_11, to_char(q2, 'FM0999999999999999')     FROM 
INT8_TBL;
--- SELECT '' AS to_char_12, to_char(q2, 'FM9999999999999999.000') FROM 
INT8_TBL;
--- SELECT '' AS to_char_13, to_char(q2, 'L9999999999999999.000')  FROM 
INT8_TBL;
--- SELECT '' AS to_char_14, to_char(q2, 'FM9999999999999999.999') FROM 
INT8_TBL;
--- SELECT '' AS to_char_15, to_char(q2, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 
9 9') FROM INT8_TBL;
--- SELECT '' AS to_char_16, to_char(q2, E'99999 "text" 9999 "9999" 999 
"\\"text between quote marks\\"" 9999') FROM INT8_TBL;
--- SELECT '' AS to_char_17, to_char(q2, '999999SG9999999999')     FROM 
INT8_TBL;
+
+SELECT '' AS to_char_1, to_char(q1, '9G999G999G999G999G999'), to_char(q2, 
'9,999,999,999,999,999')
+FROM INT8_TBL;
+
+SELECT '' AS to_char_2, to_char(q1, '9G999G999G999G999G999D999G999'), 
to_char(q2, '9,999,999,999,999,999.999,999')
+FROM INT8_TBL;
+
+SELECT '' AS to_char_3, to_char( (q1 * -1), '9999999999999999PR'), to_char( 
(q2 * -1), '9999999999999999.999PR')
+FROM INT8_TBL;
+
+SELECT '' AS to_char_4, to_char( (q1 * -1), '9999999999999999S'), to_char( (q2 
* -1), 'S9999999999999999')
+FROM INT8_TBL;
+
+SELECT '' AS to_char_5,  to_char(q2, 'MI9999999999999999')     FROM INT8_TBL;
+SELECT '' AS to_char_6,  to_char(q2, 'FMS9999999999999999')    FROM INT8_TBL;
+SELECT '' AS to_char_7,  to_char(q2, 'FM9999999999999999THPR') FROM INT8_TBL;
+SELECT '' AS to_char_8,  to_char(q2, 'SG9999999999999999th')   FROM INT8_TBL;
+SELECT '' AS to_char_9,  to_char(q2, '0999999999999999')       FROM INT8_TBL;
+SELECT '' AS to_char_10, to_char(q2, 'S0999999999999999')      FROM INT8_TBL;
+SELECT '' AS to_char_11, to_char(q2, 'FM0999999999999999')     FROM INT8_TBL;
+SELECT '' AS to_char_12, to_char(q2, 'FM9999999999999999.000') FROM INT8_TBL;
+SELECT '' AS to_char_13, to_char(q2, 'L9999999999999999.000')  FROM INT8_TBL;
+SELECT '' AS to_char_14, to_char(q2, 'FM9999999999999999.999') FROM INT8_TBL;
+SELECT '' AS to_char_15, to_char(q2, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 
9') FROM INT8_TBL;
+SELECT '' AS to_char_16, to_char(q2, E'99999 "text" 9999 "9999" 999 "\\"text 
between quote marks\\"" 9999') FROM INT8_TBL;
+SELECT '' AS to_char_17, to_char(q2, '999999SG9999999999')     FROM INT8_TBL;
 
 -- [SPARK-26218] Throw exception on overflow for integers
 -- check min/max values and overflow behavior
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql 
b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql
index 14a89d526b5..a3157d21688 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql
@@ -840,40 +840,32 @@ SELECT width_bucket(double('Infinity'), 1, 10, 10),
 
 DROP TABLE width_bucket_test;
 
--- [SPARK-28137] Missing Data Type Formatting Functions: TO_CHAR
 -- TO_CHAR()
---
--- SELECT '' AS to_char_1, to_char(val, '9G999G999G999G999G999')
---     FROM num_data;
-
--- SELECT '' AS to_char_2, to_char(val, 
'9G999G999G999G999G999D999G999G999G999G999')
---     FROM num_data;
-
--- SELECT '' AS to_char_3, to_char(val, '9999999999999999.999999999999999PR')
---     FROM num_data;
-
--- SELECT '' AS to_char_4, to_char(val, '9999999999999999.999999999999999S')
---     FROM num_data;
-
--- SELECT '' AS to_char_5,  to_char(val, 'MI9999999999999999.999999999999999') 
    FROM num_data;
--- SELECT '' AS to_char_6,  to_char(val, 
'FMS9999999999999999.999999999999999')    FROM num_data;
--- SELECT '' AS to_char_7,  to_char(val, 
'FM9999999999999999.999999999999999THPR') FROM num_data;
--- SELECT '' AS to_char_8,  to_char(val, 
'SG9999999999999999.999999999999999th')   FROM num_data;
--- SELECT '' AS to_char_9,  to_char(val, '0999999999999999.999999999999999')   
    FROM num_data;
--- SELECT '' AS to_char_10, to_char(val, 'S0999999999999999.999999999999999')  
    FROM num_data;
--- SELECT '' AS to_char_11, to_char(val, 'FM0999999999999999.999999999999999') 
    FROM num_data;
--- SELECT '' AS to_char_12, to_char(val, 'FM9999999999999999.099999999999999') 
        FROM num_data;
--- SELECT '' AS to_char_13, to_char(val, 'FM9999999999990999.990999999999999') 
        FROM num_data;
--- SELECT '' AS to_char_14, to_char(val, 'FM0999999999999999.999909999999999') 
        FROM num_data;
--- SELECT '' AS to_char_15, to_char(val, 'FM9999999990999999.099999999999999') 
        FROM num_data;
--- SELECT '' AS to_char_16, to_char(val, 'L9999999999999999.099999999999999')  
FROM num_data;
--- SELECT '' AS to_char_17, to_char(val, 'FM9999999999999999.99999999999999')  
FROM num_data;
--- SELECT '' AS to_char_18, to_char(val, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 
9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9') FROM num_data;
--- SELECT '' AS to_char_19, to_char(val, 'FMS 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 
. 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9') FROM num_data;
--- SELECT '' AS to_char_20, to_char(val, E'99999 "text" 9999 "9999" 999 
"\\"text between quote marks\\"" 9999') FROM num_data;
--- SELECT '' AS to_char_21, to_char(val, '999999SG9999999999')                 
FROM num_data;
--- SELECT '' AS to_char_22, to_char(val, 'FM9999999999999999.999999999999999') 
FROM num_data;
--- SELECT '' AS to_char_23, to_char(val, '9.999EEEE')                          
FROM num_data;
+SELECT '' AS to_char_3, to_char(val, '9999999999999999.999999999999999PR'), val
+FROM num_data;
+
+SELECT '' AS to_char_4, to_char(val, '9999999999999999.999999999999999S'), val
+FROM num_data;
+
+SELECT '' AS to_char_5,  to_char(val, 'MI9999999999999999.999999999999999'), 
val     FROM num_data;
+SELECT '' AS to_char_6,  to_char(val, 'FMS9999999999999999.999999999999999'), 
val    FROM num_data;
+SELECT '' AS to_char_7,  to_char(val, 
'FM9999999999999999.999999999999999THPR'), val FROM num_data;
+SELECT '' AS to_char_8,  to_char(val, 'SG9999999999999999.999999999999999th'), 
val   FROM num_data;
+SELECT '' AS to_char_9,  to_char(val, '0999999999999999.999999999999999'), val 
      FROM num_data;
+SELECT '' AS to_char_10, to_char(val, 'S0999999999999999.999999999999999'), 
val      FROM num_data;
+SELECT '' AS to_char_11, to_char(val, 'FM0999999999999999.999999999999999'), 
val     FROM num_data;
+SELECT '' AS to_char_12, to_char(val, 'FM9999999999999999.099999999999999'), 
val       FROM num_data;
+SELECT '' AS to_char_13, to_char(val, 'FM9999999999990999.990999999999999'), 
val       FROM num_data;
+SELECT '' AS to_char_14, to_char(val, 'FM0999999999999999.999909999999999'), 
val       FROM num_data;
+SELECT '' AS to_char_15, to_char(val, 'FM9999999990999999.099999999999999'), 
val       FROM num_data;
+SELECT '' AS to_char_16, to_char(val, 'L9999999999999999.099999999999999'), 
val        FROM num_data;
+SELECT '' AS to_char_17, to_char(val, 'FM9999999999999999.99999999999999'), 
val        FROM num_data;
+SELECT '' AS to_char_18, to_char(val, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 
9 9 9 9 9 9 9 9 9 9 9 9 9 9 9'), val FROM num_data;
+SELECT '' AS to_char_19, to_char(val, 'FMS 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 
9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9'), val FROM num_data;
+SELECT '' AS to_char_20, to_char(val, E'99999 "text" 9999 "9999" 999 "\\"text 
between quote marks\\"" 9999'), val FROM num_data;
+SELECT '' AS to_char_21, to_char(val, '999999SG9999999999'), val               
        FROM num_data;
+SELECT '' AS to_char_22, to_char(val, 'FM9999999999999999.999999999999999'), 
val       FROM num_data;
+SELECT '' AS to_char_23, to_char(val, '9.999EEEE'), val                        
        FROM num_data;
 
 -- SELECT '' AS to_char_24, to_char('100'::numeric, 'FM999.9');
 -- SELECT '' AS to_char_25, to_char('100'::numeric, 'FM999.');
diff --git 
a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out 
b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out
index 2ef77e08186..a329c0bd82c 100755
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 85
+-- Number of queries: 102
 
 
 -- !query
@@ -569,6 +569,186 @@ struct<max(q1):bigint,max(q2):bigint>
 4567890123456789       4567890123456789
 
 
+-- !query
+SELECT '' AS to_char_1, to_char(q1, '9G999G999G999G999G999'), to_char(q2, 
'9,999,999,999,999,999')
+FROM INT8_TBL
+-- !query schema
+struct<to_char_1:string,to_char(q1, 9G999G999G999G999G999):string,to_char(q2, 
9,999,999,999,999,999):string>
+-- !query output
+                         123                     456
+                         123   4,567,890,123,456,789
+       4,567,890,123,456,789                     123
+       4,567,890,123,456,789   4,567,890,123,456,789
+       4,567,890,123,456,789   4,567,890,123,456,789
+
+
+-- !query
+SELECT '' AS to_char_2, to_char(q1, '9G999G999G999G999G999D999G999'), 
to_char(q2, '9,999,999,999,999,999.999,999')
+FROM INT8_TBL
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(CAST(spark_catalog.default.int8_tbl.q1 AS 
DECIMAL(20,0)), '9G999G999G999G999G999D999G999')' due to data type mismatch: 
Thousands separators (, or G) may not appear after the decimal point in the 
number format: '9G999G999G999G999G999D999G999'; line 1 pos 24
+
+
+-- !query
+SELECT '' AS to_char_3, to_char( (q1 * -1), '9999999999999999PR'), to_char( 
(q2 * -1), '9999999999999999.999PR')
+FROM INT8_TBL
+-- !query schema
+struct<to_char_3:string,to_char((q1 * -1), 
9999999999999999PR):string,to_char((q2 * -1), 9999999999999999.999PR):string>
+-- !query output
+                    <123>                   <456.000>
+                    <123>      <4567890123456789.000>
+       <4567890123456789>                   <123.000>
+       <4567890123456789>      4567890123456789.000  
+       <4567890123456789>      <4567890123456789.000>
+
+
+-- !query
+SELECT '' AS to_char_4, to_char( (q1 * -1), '9999999999999999S'), to_char( (q2 
* -1), 'S9999999999999999')
+FROM INT8_TBL
+-- !query schema
+struct<to_char_4:string,to_char((q1 * -1), 
9999999999999999S):string,to_char((q2 * -1), S9999999999999999):string>
+-- !query output
+                    123-                    -456
+                    123-       -4567890123456789
+       4567890123456789-                    -123
+       4567890123456789-       +4567890123456789
+       4567890123456789-       -4567890123456789
+
+
+-- !query
+SELECT '' AS to_char_5,  to_char(q2, 'MI9999999999999999')     FROM INT8_TBL
+-- !query schema
+struct<to_char_5:string,to_char(q2, MI9999999999999999):string>
+-- !query output
+                     123
+                     456
+        4567890123456789
+        4567890123456789
+       -4567890123456789
+
+
+-- !query
+SELECT '' AS to_char_6,  to_char(q2, 'FMS9999999999999999')    FROM INT8_TBL
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(CAST(spark_catalog.default.int8_tbl.q2 AS 
DECIMAL(20,0)), 'FMS9999999999999999')' due to data type mismatch: Unexpected 
character 'F'' found in the format string 'FMS9999999999999999'; the structure 
of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] 
[PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_7,  to_char(q2, 'FM9999999999999999THPR') FROM INT8_TBL
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(CAST(spark_catalog.default.int8_tbl.q2 AS 
DECIMAL(20,0)), 'FM9999999999999999THPR')' due to data type mismatch: 
Unexpected character 'F'' found in the format string 'FM9999999999999999THPR'; 
the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] 
[0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_8,  to_char(q2, 'SG9999999999999999th')   FROM INT8_TBL
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(CAST(spark_catalog.default.int8_tbl.q2 AS 
DECIMAL(20,0)), 'SG9999999999999999th')' due to data type mismatch: Thousands 
separators (, or G) must have digits in between them in the number format: 
'SG9999999999999999TH'; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_9,  to_char(q2, '0999999999999999')       FROM INT8_TBL
+-- !query schema
+struct<to_char_9:string,to_char(q2, 0999999999999999):string>
+-- !query output
+       0000000000000123
+       0000000000000456
+       4567890123456789
+       4567890123456789
+       4567890123456789
+
+
+-- !query
+SELECT '' AS to_char_10, to_char(q2, 'S0999999999999999')      FROM INT8_TBL
+-- !query schema
+struct<to_char_10:string,to_char(q2, S0999999999999999):string>
+-- !query output
+       +0000000000000123
+       +0000000000000456
+       +4567890123456789
+       +4567890123456789
+       -4567890123456789
+
+
+-- !query
+SELECT '' AS to_char_11, to_char(q2, 'FM0999999999999999')     FROM INT8_TBL
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(CAST(spark_catalog.default.int8_tbl.q2 AS 
DECIMAL(20,0)), 'FM0999999999999999')' due to data type mismatch: Unexpected 
character 'F'' found in the format string 'FM0999999999999999'; the structure 
of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] 
[PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_12, to_char(q2, 'FM9999999999999999.000') FROM INT8_TBL
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(CAST(spark_catalog.default.int8_tbl.q2 AS 
DECIMAL(20,0)), 'FM9999999999999999.000')' due to data type mismatch: 
Unexpected character 'F'' found in the format string 'FM9999999999999999.000'; 
the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] 
[0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_13, to_char(q2, 'L9999999999999999.000')  FROM INT8_TBL
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(CAST(spark_catalog.default.int8_tbl.q2 AS 
DECIMAL(20,0)), 'L9999999999999999.000')' due to data type mismatch: Unexpected 
character 'L'' found in the format string 'L9999999999999999.000'; the 
structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] [0|9]* 
[$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_14, to_char(q2, 'FM9999999999999999.999') FROM INT8_TBL
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(CAST(spark_catalog.default.int8_tbl.q2 AS 
DECIMAL(20,0)), 'FM9999999999999999.999')' due to data type mismatch: 
Unexpected character 'F'' found in the format string 'FM9999999999999999.999'; 
the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] 
[0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_15, to_char(q2, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 
9') FROM INT8_TBL
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(CAST(spark_catalog.default.int8_tbl.q2 AS 
DECIMAL(20,0)), 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9')' due to data type 
mismatch: Unexpected character ' '' found in the format string 'S 9 9 9 9 9 9 9 
9 9 9 9 9 9 9 9 9 . 9 9 9'; the structure of the format string must match: 
[MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_16, to_char(q2, E'99999 "text" 9999 "9999" 999 "\\"text 
between quote marks\\"" 9999') FROM INT8_TBL
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+
+Literals of type 'E' are currently not supported.(line 1, pos 37)
+
+== SQL ==
+SELECT '' AS to_char_16, to_char(q2, E'99999 "text" 9999 "9999" 999 "\\"text 
between quote marks\\"" 9999') FROM INT8_TBL
+-------------------------------------^^^
+
+
+-- !query
+SELECT '' AS to_char_17, to_char(q2, '999999SG9999999999')     FROM INT8_TBL
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(CAST(spark_catalog.default.int8_tbl.q2 AS 
DECIMAL(20,0)), '999999SG9999999999')' due to data type mismatch: Thousands 
separators (, or G) must have digits in between them in the number format: 
'999999SG9999999999'; line 1 pos 25
+
+
 -- !query
 select bigint('9223372036854775800') / bigint('0')
 -- !query schema
diff --git 
a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out 
b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out
index 9a6cc7eac02..f0fb9434ded 100644
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 601
+-- Number of queries: 622
 
 
 -- !query
@@ -4594,6 +4594,242 @@ struct<>
 
 
 
+-- !query
+SELECT '' AS to_char_3, to_char(val, '9999999999999999.999999999999999PR'), val
+FROM num_data
+-- !query schema
+struct<to_char_3:string,to_char(val, 
9999999999999999.999999999999999PR):string,val:decimal(38,10)>
+-- !query output
+                      0.000000000000000        0.0000000000
+                      0.000000000000000        0.0000000000
+                      4.310000000000000        4.3100000000
+                  16397.038491000000000        16397.0384910000
+                  74881.000000000000000        74881.0000000000
+                  93901.577630260000000        93901.5776302600
+                7799461.411900000000000        7799461.4119000000
+               <24926804.045047420000000>      -24926804.0450474200
+               <34338492.215397047000000>      -34338492.2153970470
+               <83028485.000000000000000>      -83028485.0000000000
+
+
+-- !query
+SELECT '' AS to_char_4, to_char(val, '9999999999999999.999999999999999S'), val
+FROM num_data
+-- !query schema
+struct<to_char_4:string,to_char(val, 
9999999999999999.999999999999999S):string,val:decimal(38,10)>
+-- !query output
+                      0.000000000000000+       0.0000000000
+                      0.000000000000000+       0.0000000000
+                      4.310000000000000+       4.3100000000
+                  16397.038491000000000+       16397.0384910000
+                  74881.000000000000000+       74881.0000000000
+                  93901.577630260000000+       93901.5776302600
+                7799461.411900000000000+       7799461.4119000000
+               24926804.045047420000000-       -24926804.0450474200
+               34338492.215397047000000-       -34338492.2153970470
+               83028485.000000000000000-       -83028485.0000000000
+
+
+-- !query
+SELECT '' AS to_char_5,  to_char(val, 'MI9999999999999999.999999999999999'), 
val     FROM num_data
+-- !query schema
+struct<to_char_5:string,to_char(val, 
MI9999999999999999.999999999999999):string,val:decimal(38,10)>
+-- !query output
+                       0.000000000000000       0.0000000000
+                       0.000000000000000       0.0000000000
+                       4.310000000000000       4.3100000000
+                   16397.038491000000000       16397.0384910000
+                   74881.000000000000000       74881.0000000000
+                   93901.577630260000000       93901.5776302600
+                 7799461.411900000000000       7799461.4119000000
+               -24926804.045047420000000       -24926804.0450474200
+               -34338492.215397047000000       -34338492.2153970470
+               -83028485.000000000000000       -83028485.0000000000
+
+
+-- !query
+SELECT '' AS to_char_6,  to_char(val, 'FMS9999999999999999.999999999999999'), 
val    FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 
'FMS9999999999999999.999999999999999')' due to data type mismatch: Unexpected 
character 'F'' found in the format string 
'FMS9999999999999999.999999999999999'; the structure of the format string must 
match: [MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_7,  to_char(val, 
'FM9999999999999999.999999999999999THPR'), val FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 
'FM9999999999999999.999999999999999THPR')' due to data type mismatch: 
Unexpected character 'F'' found in the format string 
'FM9999999999999999.999999999999999THPR'; the structure of the format string 
must match: [MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_8,  to_char(val, 'SG9999999999999999.999999999999999th'), 
val   FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 
'SG9999999999999999.999999999999999th')' due to data type mismatch: Thousands 
separators (, or G) must have digits in between them in the number format: 
'SG9999999999999999.999999999999999TH'; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_9,  to_char(val, '0999999999999999.999999999999999'), val 
      FROM num_data
+-- !query schema
+struct<to_char_9:string,to_char(val, 
0999999999999999.999999999999999):string,val:decimal(38,10)>
+-- !query output
+       0000000000000000.000000000000000        0.0000000000
+       0000000000000000.000000000000000        0.0000000000
+       0000000000000004.310000000000000        4.3100000000
+       0000000000016397.038491000000000        16397.0384910000
+       0000000000074881.000000000000000        74881.0000000000
+       0000000000093901.577630260000000        93901.5776302600
+       0000000007799461.411900000000000        7799461.4119000000
+       0000000024926804.045047420000000        -24926804.0450474200
+       0000000034338492.215397047000000        -34338492.2153970470
+       0000000083028485.000000000000000        -83028485.0000000000
+
+
+-- !query
+SELECT '' AS to_char_10, to_char(val, 'S0999999999999999.999999999999999'), 
val      FROM num_data
+-- !query schema
+struct<to_char_10:string,to_char(val, 
S0999999999999999.999999999999999):string,val:decimal(38,10)>
+-- !query output
+       +0000000000000000.000000000000000       0.0000000000
+       +0000000000000000.000000000000000       0.0000000000
+       +0000000000000004.310000000000000       4.3100000000
+       +0000000000016397.038491000000000       16397.0384910000
+       +0000000000074881.000000000000000       74881.0000000000
+       +0000000000093901.577630260000000       93901.5776302600
+       +0000000007799461.411900000000000       7799461.4119000000
+       -0000000024926804.045047420000000       -24926804.0450474200
+       -0000000034338492.215397047000000       -34338492.2153970470
+       -0000000083028485.000000000000000       -83028485.0000000000
+
+
+-- !query
+SELECT '' AS to_char_11, to_char(val, 'FM0999999999999999.999999999999999'), 
val     FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 
'FM0999999999999999.999999999999999')' due to data type mismatch: Unexpected 
character 'F'' found in the format string 'FM0999999999999999.999999999999999'; 
the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] 
[0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_12, to_char(val, 'FM9999999999999999.099999999999999'), 
val       FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 
'FM9999999999999999.099999999999999')' due to data type mismatch: Unexpected 
character 'F'' found in the format string 'FM9999999999999999.099999999999999'; 
the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] 
[0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_13, to_char(val, 'FM9999999999990999.990999999999999'), 
val       FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 
'FM9999999999990999.990999999999999')' due to data type mismatch: Unexpected 
character 'F'' found in the format string 'FM9999999999990999.990999999999999'; 
the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] 
[0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_14, to_char(val, 'FM0999999999999999.999909999999999'), 
val       FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 
'FM0999999999999999.999909999999999')' due to data type mismatch: Unexpected 
character 'F'' found in the format string 'FM0999999999999999.999909999999999'; 
the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] 
[0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_15, to_char(val, 'FM9999999990999999.099999999999999'), 
val       FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 
'FM9999999990999999.099999999999999')' due to data type mismatch: Unexpected 
character 'F'' found in the format string 'FM9999999990999999.099999999999999'; 
the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] 
[0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_16, to_char(val, 'L9999999999999999.099999999999999'), 
val        FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 
'L9999999999999999.099999999999999')' due to data type mismatch: Unexpected 
character 'L'' found in the format string 'L9999999999999999.099999999999999'; 
the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] 
[0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_17, to_char(val, 'FM9999999999999999.99999999999999'), 
val        FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 
'FM9999999999999999.99999999999999')' due to data type mismatch: Unexpected 
character 'F'' found in the format string 'FM9999999999999999.99999999999999'; 
the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] 
[0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_18, to_char(val, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 
9 9 9 9 9 9 9 9 9 9 9 9 9 9 9'), val FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 'S 9 9 9 9 9 9 9 9 
9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9')' due to data type 
mismatch: Unexpected character ' '' found in the format string 'S 9 9 9 9 9 9 9 
9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9'; the structure of the 
format string must match: [MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] [PR|MI|S]; 
line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_19, to_char(val, 'FMS 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 
9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9'), val FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 'FMS 9 9 9 9 9 9 9 
9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9')' due to data type 
mismatch: Unexpected character 'F'' found in the format string 'FMS 9 9 9 9 9 9 
9 9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9'; the structure of the 
format string must match: [MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] [PR|MI|S]; 
line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_20, to_char(val, E'99999 "text" 9999 "9999" 999 "\\"text 
between quote marks\\"" 9999'), val FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+
+Literals of type 'E' are currently not supported.(line 1, pos 38)
+
+== SQL ==
+SELECT '' AS to_char_20, to_char(val, E'99999 "text" 9999 "9999" 999 "\\"text 
between quote marks\\"" 9999'), val FROM num_data
+--------------------------------------^^^
+
+
+-- !query
+SELECT '' AS to_char_21, to_char(val, '999999SG9999999999'), val               
        FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 
'999999SG9999999999')' due to data type mismatch: Thousands separators (, or G) 
must have digits in between them in the number format: '999999SG9999999999'; 
line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_22, to_char(val, 'FM9999999999999999.999999999999999'), 
val       FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, 
'FM9999999999999999.999999999999999')' due to data type mismatch: Unexpected 
character 'F'' found in the format string 'FM9999999999999999.999999999999999'; 
the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] 
[0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
+-- !query
+SELECT '' AS to_char_23, to_char(val, '9.999EEEE'), val                        
        FROM num_data
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'to_char(spark_catalog.default.num_data.val, '9.999EEEE')' due 
to data type mismatch: Unexpected character 'E'' found in the format string 
'9.999EEEE'; the structure of the format string must match: [MI|S] [$] 
[0|9|G|,]* [.|D] [0|9]* [$] [PR|MI|S]; line 1 pos 25
+
+
 -- !query
 SELECT '' AS to_number_1,  to_number('-34,338,492', '99G999G999')
 -- !query schema
@@ -4609,7 +4845,7 @@ SELECT '' AS to_number_2,  
to_number('-34,338,492.654,878', '99G999G999D999G999'
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'to_number('-34,338,492.654,878', '99G999G999D999G999')' due to 
data type mismatch: Thousands separators (,) may not appear after the decimal 
point in the number format: '99G999G999D999G999'; line 1 pos 27
+cannot resolve 'to_number('-34,338,492.654,878', '99G999G999D999G999')' due to 
data type mismatch: Thousands separators (, or G) may not appear after the 
decimal point in the number format: '99G999G999D999G999'; line 1 pos 27
 
 
 -- !query
@@ -4658,7 +4894,7 @@ SELECT '' AS to_number_15, to_number('123,000','999G')
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'to_number('123,000', '999G')' due to data type mismatch: 
Thousands separators (,) must have digits in between them in the number format: 
'999G'; line 1 pos 27
+cannot resolve 'to_number('123,000', '999G')' due to data type mismatch: 
Thousands separators (, or G) must have digits in between them in the number 
format: '999G'; line 1 pos 27
 
 
 -- !query


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to