Skip to content

Conversation

@HyukjinKwon
Copy link
Member

What changes were proposed in this pull request?

There are several tests failing due to resource-closing-related and path-related problems on Windows as below.

  • RPackageUtilsSuite:
- build an R package from a jar end to end *** FAILED *** (1 second, 625 milliseconds)
  java.io.IOException: Unable to delete file: C:\projects\spark\target\tmp\1481729427517-0\a\dep2\d\dep2-d.jar
  at org.apache.commons.io.FileUtils.forceDelete(FileUtils.java:2279)
  at org.apache.commons.io.FileUtils.cleanDirectory(FileUtils.java:1653)
  at org.apache.commons.io.FileUtils.deleteDirectory(FileUtils.java:1535)

- faulty R package shows documentation *** FAILED *** (359 milliseconds)
  java.io.IOException: Unable to delete file: C:\projects\spark\target\tmp\1481729428970-0\dep1-c.jar
  at org.apache.commons.io.FileUtils.forceDelete(FileUtils.java:2279)
  at org.apache.commons.io.FileUtils.cleanDirectory(FileUtils.java:1653)
  at org.apache.commons.io.FileUtils.deleteDirectory(FileUtils.java:1535)

- SparkR zipping works properly *** FAILED *** (47 milliseconds)
  java.util.regex.PatternSyntaxException: Unknown character property name {r} near index 4

C:\projects\spark\target\tmp\1481729429282-0

    ^
  at java.util.regex.Pattern.error(Pattern.java:1955)
  at java.util.regex.Pattern.charPropertyNodeFor(Pattern.java:2781)
  • InputOutputMetricsSuite:
- input metrics for old hadoop with coalesce *** FAILED *** (240 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics with cache and coalesce *** FAILED *** (109 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics for new Hadoop API with coalesce *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-9366ec94-dac7-4a5c-a74b-3e7594a692ab\test\InputOutputMetricsSuite.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)

- input metrics when reading text file *** FAILED *** (110 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics on records read - simple *** FAILED *** (125 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics on records read - more stages *** FAILED *** (110 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics on records - New Hadoop API *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-3f10a1a4-7820-4772-b821-25fd7523bf6f\test\InputOutputMetricsSuite.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)

- input metrics on records read with cache *** FAILED *** (93 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input read/write and shuffle read/write metrics all line up *** FAILED *** (93 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics with interleaved reads *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-2638d893-e89b-47ce-acd0-bbaeee78dd9b\InputOutputMetricsSuite_cart.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)

- input metrics with old CombineFileInputFormat *** FAILED *** (157 milliseconds)
  17947 was not greater than or equal to 300000 (InputOutputMetricsSuite.scala:324)
  org.scalatest.exceptions.TestFailedException:
  at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
  at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:466)

- input metrics with new CombineFileInputFormat *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-11920c08-19d8-4c7c-9fba-28ed72b79f80\test\InputOutputMetricsSuite.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)
  • ReplayListenerSuite:
- End-to-end replay *** FAILED *** (121 milliseconds)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)


- End-to-end replay with compression *** FAILED *** (516 milliseconds)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88)
  • EventLoggingListenerSuite:
- End-to-end event logging *** FAILED *** (7 seconds, 435 milliseconds)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88)

- End-to-end event logging with compression *** FAILED *** (1 second)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88)

- Event log name *** FAILED *** (16 milliseconds)
  "file:/[]base-dir/app1" did not equal "file:/[C:/]base-dir/app1" (EventLoggingListenerSuite.scala:123)
  org.scalatest.exceptions.TestFailedException:
  at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
  at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:466)

This PR proposes to fix the test failures on Windows

How was this patch tested?

Manually tested via AppVeyor

Before

RPackageUtilsSuite: https://ci.appveyor.com/project/spark-test/spark/build/273-RPackageUtilsSuite-before
InputOutputMetricsSuite: https://ci.appveyor.com/project/spark-test/spark/build/272-InputOutputMetricsSuite-before
ReplayListenerSuite: https://ci.appveyor.com/project/spark-test/spark/build/274-ReplayListenerSuite-before
EventLoggingListenerSuite: https://ci.appveyor.com/project/spark-test/spark/build/275-EventLoggingListenerSuite-before

After

RPackageUtilsSuite: https://ci.appveyor.com/project/spark-test/spark/build/270-RPackageUtilsSuite
InputOutputMetricsSuite: https://ci.appveyor.com/project/spark-test/spark/build/271-InputOutputMetricsSuite
ReplayListenerSuite: https://ci.appveyor.com/project/spark-test/spark/build/277-ReplayListenerSuite-after
EventLoggingListenerSuite: https://ci.appveyor.com/project/spark-test/spark/build/278-EventLoggingListenerSuite-after

if (!rPackageBuilder(rSource, printStream, verbose, RUtils.rPackages.get)) {
print(s"ERROR: Failed to build R package in $file.", printStream)
print(RJarDoc, printStream)
Utils.tryWithSafeFinally {
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actual change is as below:

Utils.tryWithSafeFinally {
  ...
} {
  jar.close()
}

// Get the relative paths for proper naming in the zip file. Note that
// the separator should always be / for according to ZIP specification.
// `relPath` here should be, for example, "/packageTest/def.R" or "/test.R".
val relPath = file.toURI.toString.replaceFirst(dir.toURI.toString.stripSuffix("/"), "")
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It should always be / according to ZIP specification (See 4.4.17 file name: (Variable) in https://pkware.cachefly.net/webdocs/casestudies/APPNOTE.TXT)

Copy link
Member Author

@HyukjinKwon HyukjinKwon Dec 16, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

cc @shivaram, could I please ask to take a look for this one? This fixes the test, SparkR zipping works properly on Windows in RPackageUtilsSuite.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah I think using / always is good. Could you write a small comment on what the toURI is accomplishing here (as opposed to the the getAbsolutePath we were using before)

Copy link
Member Author

@HyukjinKwon HyukjinKwon Dec 17, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, I thought you meant writing a comment in the codes.. :).

it just replaces the \ on Windows to /. The reason for stripSuffix is, it seems it has a trailing / when the uri is known as a directory.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You can put it in the code as well :) Something like We convert dir to URI to force / and then remove trailing / that show up for directories

Copy link
Member Author

@HyukjinKwon HyukjinKwon Dec 17, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For example,

Before

  • Windows

    val a = file.getAbsolutePath // "C:\...\tmp\1481863447985-0"
    val b = dir.getAbsolutePath  // "C:\...\tmp\1481863447985-0\test.R"
    a.replaceFirst(b, "")        // java.util.regex.PatternSyntaxException: Unknown character property name {r} near index 4

    Full exception message:

    [info]   java.util.regex.PatternSyntaxException: Unknown character property name {r} near index 4
    [info] C:\projects\spark\target\tmp\1481863447985-0
    [info]     ^
    [info]   at java.util.regex.Pattern.error(Pattern.java:1955)
    [info]   at java.util.regex.Pattern.charPropertyNodeFor(Pattern.java:2781)
    [info]   at java.util.regex.Pattern.family(Pattern.java:2736)
    [info]   at java.util.regex.Pattern.sequence(Pattern.java:2076)
    [info]   at java.util.regex.Pattern.expr(Pattern.java:1996)
    [info]   at java.util.regex.Pattern.compile(Pattern.java:1696)
    [info]   at java.util.regex.Pattern.<init>(Pattern.java:1351)
    [info]   at java.util.regex.Pattern.compile(Pattern.java:1028)
    [info]   at java.lang.String.replaceFirst(String.java:2178)
    [info]   at org.apache.spark.deploy.RPackageUtils$$anonfun$zipRLibraries$2.apply(RPackageUtils.scala:235)
    
  • Linux/Mac

    val a = file.getAbsolutePath // "/var/.../T/1481938681657-0/test.R"
    val b = dir.getAbsolutePath  // "/var/.../T/1481938681657-0"
    a.replaceFirst(b, "")        // "/test.R"

After

  • Windows

    val a = file.toURI.toString              // "file:/C:/.../tmp/1481863447985-0/test.R"
    val b = dir.toURI.toString               // "file:/C:/.../tmp/1481863447985-0/"
    a.replaceFirst(b.stripSuffix("/"),  "")  // "/test.R"
  • Linux/Mac

    val a = file.toURI.toString              // "file:/var/.../T/1481938681657-0/test.R"
    val b = dir.toURI.toString               // "file:/var/.../T/1481938681657-0/"
    a.replaceFirst(b.stripSuffix("/"),  "")  // "/test.R"

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, thanks!

@SparkQA
Copy link

SparkQA commented Dec 16, 2016

Test build #70238 has started for PR 16305 at commit 00d5322.

assert(s"file:/base-dir/app1" === EventLoggingListener.getLogPath(
Utils.resolveURI("/base-dir"), "app1", None))
assert(s"${baseDirUri.toString}/app1" === EventLoggingListener.getLogPath(
baseDirUri, "app1", None))
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

On Windows, it compares

"file:/C:/base-dir/app1" === "file:/C:/base-dir/app1"

whereas on Linux and Mac,

"file:/base-dir/app1" === "file:/base-dir/app1"

@HyukjinKwon
Copy link
Member Author

Build started: [TESTS] org.apache.spark.scheduler.EventLoggingListenerSuite PR-16305
Build started: [TESTS] org.apache.spark.scheduler.ReplayListenerSuite PR-16305
Build started: [TESTS] org.apache.spark.metrics.InputOutputMetricsSuite PR-16305
Build started: [TESTS] org.apache.spark.deploy.RPackageUtilsSuite PR-16305

Diff: master...spark-test:08CE93A8-BCB1-47D9-8683-755109827A62

@HyukjinKwon
Copy link
Member Author

cc @srowen, Could you please take a look?

@shivaram
Copy link
Contributor

Thanks @HyukjinKwon I'll take a look at this tomorrow - BTW do either you or @srowen know why we see errors of the form KeyError: -9 in Jenkins ? I also saw this on another PR [2]

...
  File "./dev/run-tests-jenkins.py", line 219, in main
    test_result_code, test_result_note = run_tests(tests_timeout)
  File "./dev/run-tests-jenkins.py", line 140, in run_tests
    test_result_note = ' * This patch **fails %s**.' % failure_note_by_errcode[test_result_code]
KeyError: -9

[2] https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/70236/console for #16290

@HyukjinKwon
Copy link
Member Author

HyukjinKwon commented Dec 16, 2016

#16307 seems too. I am not too sure but I think (IIRC) I have seen this time to time. I also want to know if these have been manually fixed by someone so far or simply it is something gone wrong in Jenkins itself. Apparently, it seems due to a problem within Jenkins though as it says this link, https://wiki.jenkins-ci.org/display/JENKINS/Spawning+processes+from+build

@HyukjinKwon
Copy link
Member Author

retest this please

@SparkQA
Copy link

SparkQA commented Dec 16, 2016

Test build #70247 has finished for PR 16305 at commit 00d5322.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member Author

retest this please

@SparkQA
Copy link

SparkQA commented Dec 16, 2016

Test build #70252 has finished for PR 16305 at commit 00d5322.

  • This patch fails PySpark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Dec 16, 2016

Test build #3506 has started for PR 16305 at commit 00d5322.

@shivaram
Copy link
Contributor

cc @shaneknapp #16305 (comment) is the comment with the Jenkins errors I was talking about

@shaneknapp
Copy link
Contributor

re @shivaram -- this is totally bizarre, and nothing is popping in to mind about what could be causing this. it's definitely not the build timeout (5 hours) or the timeout in run-tests-jenkins.py, nor are any system cleanup cron jobs running that could be responsible for this.

i'll keep poking around and trying to figure this out.

@shivaram
Copy link
Contributor

Had a minor question on the inline comment. Otherwise the RPackageUtils change LGTM

@shaneknapp
Copy link
Contributor

test this please

@SparkQA
Copy link

SparkQA commented Dec 16, 2016

Test build #70276 has finished for PR 16305 at commit 00d5322.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member Author

retest this please

@HyukjinKwon
Copy link
Member Author

Ah, thank you @shivaram and @shaneknapp.

@SparkQA
Copy link

SparkQA commented Dec 17, 2016

Test build #70286 has finished for PR 16305 at commit 00d5322.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Dec 17, 2016

Test build #70291 has finished for PR 16305 at commit 0475ac5.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@shivaram
Copy link
Contributor

Thanks @HyukjinKwon - Merging into master

@asfgit asfgit closed this in 2bc1c95 Dec 17, 2016
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
…st failures in identified ones on Windows

## What changes were proposed in this pull request?

There are several tests failing due to resource-closing-related and path-related  problems on Windows as below.

- `RPackageUtilsSuite`:

```
- build an R package from a jar end to end *** FAILED *** (1 second, 625 milliseconds)
  java.io.IOException: Unable to delete file: C:\projects\spark\target\tmp\1481729427517-0\a\dep2\d\dep2-d.jar
  at org.apache.commons.io.FileUtils.forceDelete(FileUtils.java:2279)
  at org.apache.commons.io.FileUtils.cleanDirectory(FileUtils.java:1653)
  at org.apache.commons.io.FileUtils.deleteDirectory(FileUtils.java:1535)

- faulty R package shows documentation *** FAILED *** (359 milliseconds)
  java.io.IOException: Unable to delete file: C:\projects\spark\target\tmp\1481729428970-0\dep1-c.jar
  at org.apache.commons.io.FileUtils.forceDelete(FileUtils.java:2279)
  at org.apache.commons.io.FileUtils.cleanDirectory(FileUtils.java:1653)
  at org.apache.commons.io.FileUtils.deleteDirectory(FileUtils.java:1535)

- SparkR zipping works properly *** FAILED *** (47 milliseconds)
  java.util.regex.PatternSyntaxException: Unknown character property name {r} near index 4

C:\projects\spark\target\tmp\1481729429282-0

    ^
  at java.util.regex.Pattern.error(Pattern.java:1955)
  at java.util.regex.Pattern.charPropertyNodeFor(Pattern.java:2781)
```

- `InputOutputMetricsSuite`:

```
- input metrics for old hadoop with coalesce *** FAILED *** (240 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics with cache and coalesce *** FAILED *** (109 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics for new Hadoop API with coalesce *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-9366ec94-dac7-4a5c-a74b-3e7594a692ab\test\InputOutputMetricsSuite.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)

- input metrics when reading text file *** FAILED *** (110 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics on records read - simple *** FAILED *** (125 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics on records read - more stages *** FAILED *** (110 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics on records - New Hadoop API *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-3f10a1a4-7820-4772-b821-25fd7523bf6f\test\InputOutputMetricsSuite.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)

- input metrics on records read with cache *** FAILED *** (93 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input read/write and shuffle read/write metrics all line up *** FAILED *** (93 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics with interleaved reads *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-2638d893-e89b-47ce-acd0-bbaeee78dd9b\InputOutputMetricsSuite_cart.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)

- input metrics with old CombineFileInputFormat *** FAILED *** (157 milliseconds)
  17947 was not greater than or equal to 300000 (InputOutputMetricsSuite.scala:324)
  org.scalatest.exceptions.TestFailedException:
  at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
  at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:466)

- input metrics with new CombineFileInputFormat *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-11920c08-19d8-4c7c-9fba-28ed72b79f80\test\InputOutputMetricsSuite.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)
```

- `ReplayListenerSuite`:

```
- End-to-end replay *** FAILED *** (121 milliseconds)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)

- End-to-end replay with compression *** FAILED *** (516 milliseconds)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88)
```

- `EventLoggingListenerSuite`:

```
- End-to-end event logging *** FAILED *** (7 seconds, 435 milliseconds)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88)

- End-to-end event logging with compression *** FAILED *** (1 second)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88)

- Event log name *** FAILED *** (16 milliseconds)
  "file:/[]base-dir/app1" did not equal "file:/[C:/]base-dir/app1" (EventLoggingListenerSuite.scala:123)
  org.scalatest.exceptions.TestFailedException:
  at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
  at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:466)
```

This PR proposes to fix the test failures on Windows

## How was this patch tested?

Manually tested via AppVeyor

**Before**

`RPackageUtilsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/273-RPackageUtilsSuite-before
`InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/272-InputOutputMetricsSuite-before
`ReplayListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/274-ReplayListenerSuite-before
`EventLoggingListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/275-EventLoggingListenerSuite-before

**After**

`RPackageUtilsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/270-RPackageUtilsSuite
`InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/271-InputOutputMetricsSuite
`ReplayListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/277-ReplayListenerSuite-after
`EventLoggingListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/278-EventLoggingListenerSuite-after

Author: hyukjinkwon <[email protected]>

Closes apache#16305 from HyukjinKwon/RPackageUtilsSuite-InputOutputMetricsSuite.
@HyukjinKwon
Copy link
Member Author

HyukjinKwon commented Mar 17, 2017

@shaneknapp, about #16305 (comment), I know it is a too wide guess but I have seen this -9 when someone rebases multiple times before the build is started quickly and it seems making other builds failed too. I have seen this three-ish times recently. This is a wild guess but I would like to note this just in case that it might be a clue.

@shaneknapp
Copy link
Contributor

shaneknapp commented Mar 17, 2017 via email

@HyukjinKwon HyukjinKwon deleted the RPackageUtilsSuite-InputOutputMetricsSuite branch January 2, 2018 03:43
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants