[SPARK-33690][SQL][FOLLOWUP] Escape further meta-characters in showString

### What changes were proposed in this pull request?

This is a followup PR for SPARK-33690 (#30647) .
In addition to the original PR, this PR intends to escape the following meta-characters in `Dataset#showString`.

  * `\r` (carrige ret)
  * `\f` (form feed)
  * `\b` (backspace)
  * `\u000B` (vertical tab)
  * `\u0007` (bell)

### Why are the changes needed?

To avoid breaking the layout of `Dataset#showString`.
`\u0007` does not break the layout of `Dataset#showString` but it's noisy (beeps for each row) so it should be also escaped.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Modified the existing tests.
I also build the documents and check the generated html for `sql-migration-guide.md`.

Closes #31144 from sarutak/escape-metacharacters-in-getRows.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
This commit is contained in:
Kousuke Saruta 2021-01-13 18:13:01 -06:00 committed by Sean Owen
parent 62d8466c74
commit b7da108cae
3 changed files with 46 additions and 27 deletions

View file

@ -26,7 +26,14 @@ license: |
- In Spark 3.2, `spark.sql.adaptive.enabled` is enabled by default. To restore the behavior before Spark 3.2, you can set `spark.sql.adaptive.enabled` to `false`.
- In Spark 3.2, the meta-characters `\n` and `\t` are escaped in the `show()` action. In Spark 3.1 or earlier, the two metacharacters are output as it is.
- In Spark 3.2, the following meta-characters are escaped in the `show()` action. In Spark 3.1 or earlier, the following metacharacters are output as it is.
* `\n` (new line)
* `\r` (carrige ret)
* `\t` (horizontal tab)
* `\f` (form feed)
* `\b` (backspace)
* `\u000B` (vertical tab)
* `\u0007` (bell)
- In Spark 3.2, `ALTER TABLE .. RENAME TO PARTITION` throws `PartitionAlreadyExistsException` instead of `AnalysisException` for tables from Hive external when the target partition already exists.

View file

@ -300,17 +300,27 @@ class Dataset[T] private[sql](
}
val data = newDf.select(castCols: _*).take(numRows + 1)
def escapeMetaCharacters(str: String): String = {
str.replaceAll("\n", "\\\\n")
.replaceAll("\r", "\\\\r")
.replaceAll("\t", "\\\\t")
.replaceAll("\f", "\\\\f")
.replaceAll("\b", "\\\\b")
.replaceAll("\u000B", "\\\\v")
.replaceAll("\u0007", "\\\\a")
}
// For array values, replace Seq and Array with square brackets
// For cells that are beyond `truncate` characters, replace it with the
// first `truncate-3` and "..."
schema.fieldNames.toSeq +: data.map { row =>
schema.fieldNames.map(escapeMetaCharacters).toSeq +: data.map { row =>
row.toSeq.map { cell =>
val str = cell match {
case null => "null"
case binary: Array[Byte] => binary.map("%02X".format(_)).mkString("[", " ", "]")
case _ =>
// Escapes meta-characters not to break the `showString` format
cell.toString.replaceAll("\n", "\\\\n").replaceAll("\t", "\\\\t")
escapeMetaCharacters(cell.toString)
}
if (truncate > 0 && str.length > truncate) {
// do not show ellipses for strings shorter than 4 characters.

View file

@ -1236,40 +1236,42 @@ class DataFrameSuite extends QueryTest
}
test("SPARK-33690: showString: escape meta-characters") {
val df1 = Seq("aaa\nbbb\tccc").toDF("value")
val df1 = spark.sql("SELECT 'aaa\nbbb\tccc\rddd\feee\bfff\u000Bggg\u0007hhh'")
assert(df1.showString(1, truncate = 0) ===
"""+-------------+
||value |
|+-------------+
||aaa\nbbb\tccc|
|+-------------+
"""+--------------------------------------+
||aaa\nbbb\tccc\rddd\feee\bfff\vggg\ahhh|
|+--------------------------------------+
||aaa\nbbb\tccc\rddd\feee\bfff\vggg\ahhh|
|+--------------------------------------+
|""".stripMargin)
val df2 = Seq(Seq("aaa\nbbb\tccc")).toDF("value")
val df2 = spark.sql("SELECT array('aaa\nbbb\tccc\rddd\feee\bfff\u000Bggg\u0007hhh')")
assert(df2.showString(1, truncate = 0) ===
"""+---------------+
||value |
|+---------------+
||[aaa\nbbb\tccc]|
|+---------------+
"""+---------------------------------------------+
||array(aaa\nbbb\tccc\rddd\feee\bfff\vggg\ahhh)|
|+---------------------------------------------+
||[aaa\nbbb\tccc\rddd\feee\bfff\vggg\ahhh] |
|+---------------------------------------------+
|""".stripMargin)
val df3 = Seq(Map("aaa\nbbb\tccc" -> "aaa\nbbb\tccc")).toDF("value")
val df3 =
spark.sql("SELECT map('aaa\nbbb\tccc', 'aaa\nbbb\tccc\rddd\feee\bfff\u000Bggg\u0007hhh')")
assert(df3.showString(1, truncate = 0) ===
"""+--------------------------------+
||value |
|+--------------------------------+
||{aaa\nbbb\tccc -> aaa\nbbb\tccc}|
|+--------------------------------+
"""+----------------------------------------------------------+
||map(aaa\nbbb\tccc, aaa\nbbb\tccc\rddd\feee\bfff\vggg\ahhh)|
|+----------------------------------------------------------+
||{aaa\nbbb\tccc -> aaa\nbbb\tccc\rddd\feee\bfff\vggg\ahhh} |
|+----------------------------------------------------------+
|""".stripMargin)
val df4 = Seq("aaa\nbbb\tccc").toDF("value").selectExpr("named_struct('v', value)")
val df4 =
spark.sql("SELECT named_struct('v', 'aaa\nbbb\tccc\rddd\feee\bfff\u000Bggg\u0007hhh')")
assert(df4.showString(1, truncate = 0) ===
"""+----------------------+
||named_struct(v, value)|
|+----------------------+
||{aaa\nbbb\tccc} |
|+----------------------+
"""+-------------------------------------------------------+
||named_struct(v, aaa\nbbb\tccc\rddd\feee\bfff\vggg\ahhh)|
|+-------------------------------------------------------+
||{aaa\nbbb\tccc\rddd\feee\bfff\vggg\ahhh} |
|+-------------------------------------------------------+
|""".stripMargin)
}