Skip to content

Commit a8e9834

Browse files
committed
[SPARK-53090][CORE][SS][TESTS] Use Java OutputStream.write instead of IOUtils.write
### What changes were proposed in this pull request? This PR aims to use Java `OutputStream.write` instead of `IOUtils.write`. ### Why are the changes needed? To use a better implementation for our use cases. ```scala scala> val s = "a".repeat(400_000_000) scala> spark.time(new java.io.FileOutputStream("/tmp/a").write(s.getBytes())) Time taken: 270 ms scala> spark.time(org.apache.commons.io.IOUtils.write(s, new java.io.FileOutputStream("/tmp/a"))) Time taken: 1070 ms ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #51804 from dongjoon-hyun/SPARK-53090. Authored-by: Dongjoon Hyun <dongjoon@apache.org> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
1 parent c23ce68 commit a8e9834

File tree

3 files changed

+7
-4
lines changed

3 files changed

+7
-4
lines changed

core/src/test/scala/org/apache/spark/util/UtilsSuite.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,6 @@ import scala.collection.mutable.ListBuffer
3232
import scala.util.{Random, Try}
3333

3434
import com.google.common.io.Files
35-
import org.apache.commons.io.IOUtils
3635
import org.apache.commons.math3.stat.inference.ChiSquareTest
3736
import org.apache.hadoop.conf.Configuration
3837
import org.apache.hadoop.fs.Path
@@ -343,7 +342,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties {
343342
} else {
344343
new FileOutputStream(path)
345344
}
346-
IOUtils.write(content, outputStream)
345+
outputStream.write(content)
347346
outputStream.close()
348347
content.length
349348
}

scalastyle-config.xml

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -627,6 +627,11 @@ This file is divided into 3 sections:
627627
<customMessage>Use toString of SparkStreamUtils or Utils instead.</customMessage>
628628
</check>
629629

630+
<check customId="ioutilswrite" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
631+
<parameters><parameter name="regex">\bIOUtils\.write\b</parameter></parameters>
632+
<customMessage>Use Java `write` instead.</customMessage>
633+
</check>
634+
630635
<check customId="maputils" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
631636
<parameters><parameter name="regex">org\.apache\.commons\.collections4\.MapUtils\b</parameter></parameters>
632637
<customMessage>Use org.apache.spark.util.collection.Utils instead.</customMessage>

streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@ import scala.collection.mutable
2727
import scala.jdk.CollectionConverters._
2828

2929
import com.google.common.io.Files
30-
import org.apache.commons.io.IOUtils
3130
import org.apache.hadoop.fs.{FileSystem, Path}
3231
import org.apache.hadoop.io.{LongWritable, Text}
3332
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
@@ -264,7 +263,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
264263

265264
def write(path: Path, text: String): Unit = {
266265
val out = fs.create(path, true)
267-
IOUtils.write(text, out, StandardCharsets.UTF_8)
266+
out.write(text.getBytes(StandardCharsets.UTF_8))
268267
out.close()
269268
}
270269

0 commit comments

Comments
 (0)