-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-34790][CORE] Disable fetching shuffle blocks in batch when io encryption is enabled #31898
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
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 |
|---|---|---|
|
|
@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution | |
| import org.scalatest.BeforeAndAfterAll | ||
|
|
||
| import org.apache.spark.{SparkConf, SparkFunSuite} | ||
| import org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED | ||
| import org.apache.spark.internal.config.UI.UI_ENABLED | ||
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.execution.adaptive._ | ||
|
|
@@ -57,15 +58,18 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl | |
| def withSparkSession( | ||
| f: SparkSession => Unit, | ||
| targetPostShuffleInputSize: Int, | ||
| minNumPostShufflePartitions: Option[Int]): Unit = { | ||
| minNumPostShufflePartitions: Option[Int], | ||
| enableIOEncryption: Boolean = false): Unit = { | ||
| val sparkConf = | ||
| new SparkConf(false) | ||
| .setMaster("local[*]") | ||
| .setAppName("test") | ||
| .set(UI_ENABLED, false) | ||
| .set(IO_ENCRYPTION_ENABLED, enableIOEncryption) | ||
| .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") | ||
| .set(SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key, "5") | ||
| .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") | ||
| .set(SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key, "true") | ||
| .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") | ||
| .set( | ||
| SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key, | ||
|
|
@@ -408,6 +412,25 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl | |
| } | ||
| withSparkSession(test, 100, None) | ||
| } | ||
|
|
||
| test("SPARK-34790: enable IO encryption in AQE partition coalescing") { | ||
|
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. I can't reproduce the issue by this test. @hezuojiao Could you double-check it?
Contributor
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 can reproduce the issue because my laptop has only 4 cores :) |
||
| val test: SparkSession => Unit = { spark: SparkSession => | ||
| val ds = spark.range(0, 100, 1, numInputPartitions) | ||
| val resultDf = ds.repartition(ds.col("id")) | ||
| resultDf.collect() | ||
|
|
||
| val finalPlan = resultDf.queryExecution.executedPlan | ||
| .asInstanceOf[AdaptiveSparkPlanExec].executedPlan | ||
| assert( | ||
| finalPlan.collect { | ||
| case r @ CoalescedShuffleReader() => r | ||
| }.isDefinedAt(0)) | ||
| } | ||
| Seq(true, false).foreach { enableIOEncryption => | ||
| // Before SPARK-34790, it will throw an exception when io encryption enabled. | ||
| withSparkSession(test, Int.MaxValue, None, enableIOEncryption) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| object CoalescedShuffleReader { | ||
|
|
||
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.
I think we still have enough time to take a look and make AQE supported with IO encryption. Does it require a lot of change to make it supported?
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.
Yes, you are right. This PR makes a simple read control, which may sacrifice shuffle read performance in AQE. It would be better to allow batch fetching for contiguous shuffle blocks while IO encryption enabled.
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.
BTW, it seems to me that when io encryption enabled, the demarcation point (end of blocks) between consecutive blocks cannot be resolved correctly during deserialization (shuffle read).
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.
We could store the size info of each consecutive block in
ShuffleBlockBatchIdto resolve it?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.
That sounds like a good idea but we need to think about how to support it in the current shuffle protocol.
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.
Maybe we can have this bandage fix first. The encryption shuffle mode breaks the assumption of batch serialized and deserialize, similar to the issue with codec concatenation.