Skip to content

[BLAZE-1071][FOLLOWUP] Fix incorrect mapStatus to prevent Uniffle failures in Blaze#1072

Merged
richox merged 2 commits intoapache:masterfrom
work-space-station:blaze-1071
Jul 29, 2025
Merged

[BLAZE-1071][FOLLOWUP] Fix incorrect mapStatus to prevent Uniffle failures in Blaze#1072
richox merged 2 commits intoapache:masterfrom
work-space-station:blaze-1071

Conversation

@merrily01
Copy link
Member

@merrily01 merrily01 commented Jul 26, 2025

Which issue does this PR close?

This is the followup of #901 and #979

Closes #1071

Rationale for this change

When using Uniffle as the shuffle service in Blaze, the constructed MapStatus in BlazeRssShuffleWriterBase may lack the required topologyInfo in its BlockManagerId.
This causes Uniffle to throw an error during shuffle read:

Can't get expected taskAttemptId

The root cause is that although Uniffle internally builds a MapStatus with proper BlockManagerId (including topologyInfo), this status is not propagated or reused by Blaze. Instead, Blaze re-constructs a new MapStatus using SparkEnv.get.blockManager.shuffleServerId, which lacks the correct topologyInfo.

This PR ensures that the correct MapStatus (with valid topologyInfo) returned by Uniffle’s shuffle writer is preserved and returned to the shuffle system, avoiding this mismatch and runtime failure.

What changes are included in this PR?

  • Update BlazeRssShuffleWriterBase.stop() to prefer the MapStatus returned by the underlying RSS shuffle writer (e.g., Uniffle).

  • Fallback to the original construction logic only if no MapStatus is returned.

Are there any user-facing changes?

No.

@merrily01
Copy link
Member Author

PTAL @richox @zuston

@richox richox merged commit fe59e57 into apache:master Jul 29, 2025
619 checks passed
@zuston
Copy link
Member

zuston commented Jul 29, 2025

Sorry for the late review. LTGM

}

override def rssStop(success: Boolean): Unit = {
override def rssStop(success: Boolean): Option[MapStatus] = {
Copy link
Contributor

Choose a reason for hiding this comment

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

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Missing topologyInfo in MapStatus causes Uniffle failure in Blaze

4 participants