Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
21 changes: 21 additions & 0 deletions core/src/test/scala/org/apache/spark/FileSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,12 @@ package org.apache.spark

import java.io._
import java.nio.ByteBuffer
import java.nio.charset.StandardCharsets
import java.util.zip.GZIPOutputStream

import scala.io.Source

import com.google.common.io.Files
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.io._
Expand Down Expand Up @@ -299,6 +301,25 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
}
}

test("SPARK-22357 test binaryFiles minPartitions") {
sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")
.set("spark.files.openCostInBytes", "0")
Copy link
Contributor

Choose a reason for hiding this comment

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

why is this setting needed: spark.files.openCostInBytes

Copy link
Member Author

Choose a reason for hiding this comment

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

This removes its effect in the section of code we're really trying to test:

def setMinPartitions(sc: SparkContext, context: JobContext, minPartitions: Int) {
    val defaultMaxSplitBytes = sc.getConf.get(config.FILES_MAX_PARTITION_BYTES)
    val openCostInBytes = sc.getConf.get(config.FILES_OPEN_COST_IN_BYTES)
    val defaultParallelism = Math.max(sc.defaultParallelism, minPartitions)
    val files = listStatus(context).asScala
    val totalBytes = files.filterNot(_.isDirectory).map(_.getLen + openCostInBytes).sum
    val bytesPerCore = totalBytes / defaultParallelism
    val maxSplitSize = Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore))
    super.setMaxSplitSize(maxSplitSize)
  }

Copy link
Contributor

Choose a reason for hiding this comment

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

ah, I see, thanks for pointing that out!

.set("spark.default.parallelism", "1"))

val tempDir = Utils.createTempDir()
val tempDirPath = tempDir.getAbsolutePath

for (i <- 0 until 8) {
val tempFile = new File(tempDir, s"part-0000$i")
Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", tempFile,
StandardCharsets.UTF_8)
}

for (p <- Seq(1, 2, 8)) {
assert(sc.binaryFiles(tempDirPath, minPartitions = p).getNumPartitions === p)
}
}

test("fixed record length binary file as byte array") {
sc = new SparkContext("local", "test")
val testOutput = Array[Byte](1, 2, 3, 4, 5, 6)
Expand Down