-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-21764][TESTS] Fix tests failures on Windows: resources not being closed and incorrect paths #18971
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-21764][TESTS] Fix tests failures on Windows: resources not being closed and incorrect paths #18971
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -831,7 +831,7 @@ class SparkSubmitSuite | |
| val hadoopConf = new Configuration() | ||
| val tmpDir = Files.createTempDirectory("tmp").toFile | ||
| updateConfWithFakeS3Fs(hadoopConf) | ||
| val sourcePath = s"s3a://${jarFile.getAbsolutePath}" | ||
| val sourcePath = s"s3a://${jarFile.toURI.getPath}" | ||
|
||
| val outputPath = DependencyUtils.downloadFile(sourcePath, tmpDir, sparkConf, hadoopConf, | ||
| new SecurityManager(sparkConf)) | ||
| checkDownloadedFile(sourcePath, outputPath) | ||
|
|
@@ -847,7 +847,7 @@ class SparkSubmitSuite | |
| val hadoopConf = new Configuration() | ||
| val tmpDir = Files.createTempDirectory("tmp").toFile | ||
| updateConfWithFakeS3Fs(hadoopConf) | ||
| val sourcePaths = Seq("/local/file", s"s3a://${jarFile.getAbsolutePath}") | ||
| val sourcePaths = Seq("/local/file", s"s3a://${jarFile.toURI.getPath}") | ||
| val outputPaths = DependencyUtils | ||
| .downloadFileList(sourcePaths.mkString(","), tmpDir, sparkConf, hadoopConf, | ||
| new SecurityManager(sparkConf)) | ||
|
|
||
| Original file line number | Diff line number | Diff line change | ||||||
|---|---|---|---|---|---|---|---|---|
|
|
@@ -21,6 +21,7 @@ import java.io._ | |||||||
| import java.net.URI | ||||||||
| import java.util.concurrent.atomic.AtomicInteger | ||||||||
|
|
||||||||
| import org.apache.hadoop.fs.Path | ||||||||
| import org.json4s.jackson.JsonMethods._ | ||||||||
| import org.scalatest.BeforeAndAfter | ||||||||
|
|
||||||||
|
|
@@ -84,24 +85,23 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp | |||||||
| val buffered = new ByteArrayOutputStream | ||||||||
| val codec = new LZ4CompressionCodec(new SparkConf()) | ||||||||
| val compstream = codec.compressedOutputStream(buffered) | ||||||||
| val writer = new PrintWriter(compstream) | ||||||||
| Utils.tryWithResource(new PrintWriter(compstream)) { writer => | ||||||||
|
|
||||||||
| val applicationStart = SparkListenerApplicationStart("AppStarts", None, | ||||||||
| 125L, "Mickey", None) | ||||||||
| val applicationEnd = SparkListenerApplicationEnd(1000L) | ||||||||
| val applicationStart = SparkListenerApplicationStart("AppStarts", None, | ||||||||
| 125L, "Mickey", None) | ||||||||
| val applicationEnd = SparkListenerApplicationEnd(1000L) | ||||||||
|
|
||||||||
| // scalastyle:off println | ||||||||
| writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) | ||||||||
| writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) | ||||||||
| // scalastyle:on println | ||||||||
| writer.close() | ||||||||
| // scalastyle:off println | ||||||||
| writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) | ||||||||
| writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) | ||||||||
| // scalastyle:on println | ||||||||
| } | ||||||||
|
|
||||||||
| val logFilePath = Utils.getFilePath(testDir, "events.lz4.inprogress") | ||||||||
| val fstream = fileSystem.create(logFilePath) | ||||||||
| val bytes = buffered.toByteArray | ||||||||
|
|
||||||||
| fstream.write(bytes, 0, buffered.size) | ||||||||
| fstream.close | ||||||||
| Utils.tryWithResource(fileSystem.create(logFilePath)) { fstream => | ||||||||
| fstream.write(bytes, 0, buffered.size) | ||||||||
| } | ||||||||
|
|
||||||||
| // Read the compressed .inprogress file and verify only first event was parsed. | ||||||||
| val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) | ||||||||
|
|
@@ -112,17 +112,19 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp | |||||||
|
|
||||||||
| // Verify the replay returns the events given the input maybe truncated. | ||||||||
| val logData = EventLoggingListener.openEventLog(logFilePath, fileSystem) | ||||||||
| val failingStream = new EarlyEOFInputStream(logData, buffered.size - 10) | ||||||||
| replayer.replay(failingStream, logFilePath.toString, true) | ||||||||
| Utils.tryWithResource(new EarlyEOFInputStream(logData, buffered.size - 10)) { failingStream => | ||||||||
|
||||||||
| replayer.replay(failingStream, logFilePath.toString, true) | ||||||||
|
|
||||||||
| assert(eventMonster.loggedEvents.size === 1) | ||||||||
| assert(failingStream.didFail) | ||||||||
| assert(eventMonster.loggedEvents.size === 1) | ||||||||
| assert(failingStream.didFail) | ||||||||
| } | ||||||||
|
|
||||||||
| // Verify the replay throws the EOF exception since the input may not be truncated. | ||||||||
| val logData2 = EventLoggingListener.openEventLog(logFilePath, fileSystem) | ||||||||
| val failingStream2 = new EarlyEOFInputStream(logData2, buffered.size - 10) | ||||||||
| intercept[EOFException] { | ||||||||
| replayer.replay(failingStream2, logFilePath.toString, false) | ||||||||
| Utils.tryWithResource(new EarlyEOFInputStream(logData2, buffered.size - 10)) { failingStream2 => | ||||||||
|
||||||||
| intercept[EOFException] { | ||||||||
| replayer.replay(failingStream2, logFilePath.toString, false) | ||||||||
| } | ||||||||
| } | ||||||||
| } | ||||||||
|
|
||||||||
|
|
@@ -151,7 +153,10 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp | |||||||
| * assumption that the event logging behavior is correct (tested in a separate suite). | ||||||||
| */ | ||||||||
| private def testApplicationReplay(codecName: Option[String] = None) { | ||||||||
| val logDirPath = Utils.getFilePath(testDir, "test-replay") | ||||||||
| val logDir = new File(testDir.getAbsolutePath, "test-replay") | ||||||||
| // Here, it creates `Path` from the URI instead of the absolute path for the explicit file | ||||||||
| // scheme so that the string representation of this `Path` has leading file scheme correctly. | ||||||||
| val logDirPath = new Path(logDir.toURI) | ||||||||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we create this from the absolute path, it appears that the string ends up with It looks For example, with
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This test itself was added long time ago but looks there was a recent change related with this code path - edcb878.
I think this simple test describes what I intended: Before scala> import org.apache.hadoop.fs.Path
import org.apache.hadoop.fs.Path
scala> val path = new Path("C:\\a\\b\\c")
path: org.apache.hadoop.fs.Path = C:/a/b/c
scala> path.toString
res0: String = C:/a/b/c
scala> path.toUri.toString
res1: String = /C:/a/b/cAfter scala> import org.apache.hadoop.fs.Path
import org.apache.hadoop.fs.Path
scala> import java.io.File
import java.io.File
scala> val file = new File("C:\\a\\b\\c")
file: java.io.File = C:\a\b\c
scala> val path = new Path(file.toURI)
path: org.apache.hadoop.fs.Path = file:/C:/a/b/c
scala> path.toString
res2: String = file:/C:/a/b/c
scala> path.toUri.toString
res3: String = file:/C:/a/b/cPlease correct me if I am mistaken here.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. cc @vanzin, I believe I need your look. Could you take a look when you have some time?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. cc @sarutak too. Other changes should be fine as they are what I have usually fixed but I am less sure of this one. Current status conservatively fixes the test only but I guess I need a sign-off on this.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. From your description it sounds like The first two should return the same thing ("file:/C:/WINDOWS" or something along those lines) while the third I'm not sure, since it's ambiguous. But that's probably the cause of the change of behavior. Anyway the code change looks correct.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I have Windows one set up properly for dev env, and also a set of scripts to run a specific Scala tests by test-only via AppVeyor automatically against a PR. So, it is not really hard to test. I am fine with asking more cases in the future. println("=== org.apache.spark.util.Utils.resolveURI")
println(Utils.resolveURI("C:\\WINDOWS").toString)
println(Utils.resolveURI("/C:/WINDOWS").toString)
println(Utils.resolveURI("C:/WINDOWS").toString)
println
println(Utils.resolveURI("C:\\WINDOWS").getScheme)
println(Utils.resolveURI("/C:/WINDOWS").getScheme)
println(Utils.resolveURI("C:/WINDOWS").getScheme)
println
println("=== java.io.File")
println(new File("C:\\WINDOWS").toURI.toString)
println(new File("/C:/WINDOWS").toURI.toString)
println(new File("C:/WINDOWS").toURI.toString)
println
println(new File("C:\\WINDOWS").toURI.getScheme)
println(new File("/C:/WINDOWS").toURI.getScheme)
println(new File("C:/WINDOWS").toURI.getScheme)
println
println("=== org.apache.hadoop.fs.Path")
println(new Path("C:\\WINDOWS").toUri.toString)
println(new Path("/C:/WINDOWS").toUri.toString)
println(new Path("C:/WINDOWS").toUri.toString)
println
println(new Path("C:\\WINDOWS").toString)
println(new Path("/C:/WINDOWS").toString)
println(new Path("C:/WINDOWS").toString)
println
println(new Path("C:\\WINDOWS").toUri.getScheme)
println(new Path("/C:/WINDOWS").toUri.getScheme)
println(new Path("C:/WINDOWS").toUri.getScheme)
println
println("=== java.io.File.toURI and org.apache.hadoop.fs.Path")
println(new Path(new File("C:\\WINDOWS").toURI).toUri.toString)
println(new Path(new File("/C:/WINDOWS").toURI).toUri.toString)
println(new Path(new File("C:/WINDOWS").toURI).toUri.toString)
println
println(new Path(new File("C:\\WINDOWS").toURI).toString)
println(new Path(new File("/C:/WINDOWS").toURI).toString)
println(new Path(new File("C:/WINDOWS").toURI).toString)
println
println(new Path(new File("C:\\WINDOWS").toURI).toUri.getScheme)
println(new Path(new File("/C:/WINDOWS").toURI).toUri.getScheme)
println(new Path(new File("C:/WINDOWS").toURI).toUri.getScheme)produced
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @HyukjinKwon I think this change it self looks reasonable. |
||||||||
| fileSystem.mkdirs(logDirPath) | ||||||||
|
|
||||||||
| val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName) | ||||||||
|
|
@@ -221,12 +226,14 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp | |||||||
| def didFail: Boolean = countDown.get == 0 | ||||||||
|
|
||||||||
| @throws[IOException] | ||||||||
| def read: Int = { | ||||||||
| override def read(): Int = { | ||||||||
| if (countDown.get == 0) { | ||||||||
| throw new EOFException("Stream ended prematurely") | ||||||||
| } | ||||||||
| countDown.decrementAndGet() | ||||||||
| in.read | ||||||||
| in.read() | ||||||||
| } | ||||||||
|
|
||||||||
| override def close(): Unit = in.close() | ||||||||
|
||||||||
| } | ||||||||
| } | ||||||||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -203,7 +203,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto | |
| sql(s"INSERT INTO TABLE $tableName PARTITION (ds='$ds') SELECT * FROM src") | ||
| } | ||
|
|
||
| sql(s"ALTER TABLE $tableName SET LOCATION '$path'") | ||
| sql(s"ALTER TABLE $tableName SET LOCATION '${path.toURI}'") | ||
|
||
|
|
||
| sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS noscan") | ||
|
|
||
|
|
@@ -222,7 +222,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto | |
| s""" | ||
| |CREATE TABLE $sourceTableName (key STRING, value STRING) | ||
| |PARTITIONED BY (ds STRING) | ||
| |LOCATION '$path' | ||
| |LOCATION '${path.toURI}' | ||
| """.stripMargin) | ||
|
|
||
| val partitionDates = List("2010-01-01", "2010-01-02", "2010-01-03") | ||
|
|
@@ -239,7 +239,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto | |
| s""" | ||
| |CREATE TABLE $tableName (key STRING, value STRING) | ||
| |PARTITIONED BY (ds STRING) | ||
| |LOCATION '$path' | ||
| |LOCATION '${path.toURI}' | ||
| """.stripMargin) | ||
|
|
||
| // Register only one of the partitions found on disk | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Simply closes
JarFile. This should be closed