Skip to content

Commit

Permalink
[SPARK-48757][CORE] Make IndexShuffleBlockResolver have explicit co…
Browse files Browse the repository at this point in the history
…nstructors

### What changes were proposed in this pull request?

This PR aims to make `IndexShuffleBlockResolver` have explicit constructors from Apache Spark 4.0.0.

### Why are the changes needed?

Although `IndexShuffleBlockResolver` is `private` and there is no contract to keep class constructor signatures, from Apache Spark 4.0.0, we had better reduce the following situations with the old libraries built against old Spark versions.

```
Cause: java.lang.NoSuchMethodError: 'void org.apache.spark.shuffle.IndexShuffleBlockResolver.<init>(org.apache.spark.SparkConf, org.apache.spark.storage.BlockManager)'
[info] at org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager.<init>(CometShuffleManager.scala:64)
```

### 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 apache#47148 from dongjoon-hyun/SPARK-48757.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
  • Loading branch information
dongjoon-hyun committed Jun 28, 2024
1 parent 4e57f06 commit 6bfeb09
Showing 1 changed file with 14 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -57,11 +57,23 @@ import org.apache.spark.util.collection.OpenHashSet
private[spark] class IndexShuffleBlockResolver(
conf: SparkConf,
// var for testing
var _blockManager: BlockManager = null,
val taskIdMapsForShuffle: JMap[Int, OpenHashSet[Long]] = Collections.emptyMap())
var _blockManager: BlockManager,
val taskIdMapsForShuffle: JMap[Int, OpenHashSet[Long]])
extends ShuffleBlockResolver
with Logging with MigratableResolver {

def this(conf: SparkConf) = {
this(conf, null, Collections.emptyMap())
}

def this(conf: SparkConf, _blockManager: BlockManager) = {
this(conf, _blockManager, Collections.emptyMap())
}

def this(conf: SparkConf, taskIdMapsForShuffle: JMap[Int, OpenHashSet[Long]]) = {
this(conf, null, taskIdMapsForShuffle)
}

private lazy val blockManager = Option(_blockManager).getOrElse(SparkEnv.get.blockManager)

private val transportConf = {
Expand Down

0 comments on commit 6bfeb09

Please sign in to comment.