Skip to content
Snippets Groups Projects
Unverified Commit f1330b1d authored by hyukjinkwon's avatar hyukjinkwon Committed by Sean Owen
Browse files

[SPARK-19022][TESTS] Fix tests dependent on OS due to different newline characters

## What changes were proposed in this pull request?

There are two tests failing on Windows due to the different newlines.

```
 - StreamingQueryProgress - prettyJson *** FAILED *** (0 milliseconds)
 "{
    "id" : "39788670-6722-48b7-a248-df6ba08722ac",
    "runId" : "422282f1-3b81-4b47-a15d-82dda7e69390",
    "name" : "myName",
    ...
  }" did not equal "{
    "id" : "39788670-6722-48b7-a248-df6ba08722ac",
    "runId" : "422282f1-3b81-4b47-a15d-82dda7e69390",
    "name" : "myName",
    ...
  }"
  ...
```

```
 - StreamingQueryStatus - prettyJson *** FAILED *** (0 milliseconds)
 "{
    "message" : "active",
    "isDataAvailable" : true,
    "isTriggerActive" : false
  }" did not equal "{
    "message" : "active",
    "isDataAvailable" : true,
    "isTriggerActive" : false
  }"
  ...
```

The reason is, `pretty` in `org.json4s.pretty` writes OS-dependent newlines but the string defined in the tests are `\n`. This ends up with test failures.

This PR proposes to compare these regardless of newline concerns.

## How was this patch tested?

Manually tested via AppVeyor.

**Before**
https://ci.appveyor.com/project/spark-test/spark/build/417-newlines-fix-before

**After**
https://ci.appveyor.com/project/spark-test/spark/build/418-newlines-fix

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16433 from HyukjinKwon/tests-StreamingQueryStatusAndProgressSuite.
parent 0ac2f1e7
No related branches found
No related tags found
No related merge requests found
......@@ -30,10 +30,16 @@ import org.apache.spark.sql.streaming.StreamingQueryStatusAndProgressSuite._
class StreamingQueryStatusAndProgressSuite extends StreamTest {
implicit class EqualsIgnoreCRLF(source: String) {
def equalsIgnoreCRLF(target: String): Boolean = {
source.replaceAll("\r\n|\r|\n", System.lineSeparator) ===
target.replaceAll("\r\n|\r|\n", System.lineSeparator)
}
}
test("StreamingQueryProgress - prettyJson") {
val json1 = testProgress1.prettyJson
assert(json1 ===
assert(json1.equalsIgnoreCRLF(
s"""
|{
| "id" : "${testProgress1.id.toString}",
......@@ -66,12 +72,12 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest {
| "description" : "sink"
| }
|}
""".stripMargin.trim)
""".stripMargin.trim))
assert(compact(parse(json1)) === testProgress1.json)
val json2 = testProgress2.prettyJson
assert(
json2 ===
json2.equalsIgnoreCRLF(
s"""
|{
| "id" : "${testProgress2.id.toString}",
......@@ -96,7 +102,7 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest {
| "description" : "sink"
| }
|}
""".stripMargin.trim)
""".stripMargin.trim))
assert(compact(parse(json2)) === testProgress2.json)
}
......@@ -112,14 +118,14 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest {
test("StreamingQueryStatus - prettyJson") {
val json = testStatus.prettyJson
assert(json ===
assert(json.equalsIgnoreCRLF(
"""
|{
| "message" : "active",
| "isDataAvailable" : true,
| "isTriggerActive" : false
|}
""".stripMargin.trim)
""".stripMargin.trim))
}
test("StreamingQueryStatus - json") {
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment